blob: 91edc52a50d11f77dab969d221c1a57ed017becd [file] [log] [blame]
<!DOCTYPE html>
<html lang="en">
<head>
<meta charset="UTF-8">
<!--[if IE]><meta http-equiv="X-UA-Compatible" content="IE=edge"><![endif]-->
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="generator" content="Asciidoctor 1.5.5">
<meta name="author" content="Apache HBase Team">
<title>Apache HBase &#8482; Reference Guide</title>
<link rel="stylesheet" href="./hbase.css">
<link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/font-awesome/4.6.3/css/font-awesome.min.css">
<link rel="stylesheet" href="./coderay-asciidoctor.css">
</head>
<body class="book toc2 toc-left">
<div id="header">
<h1>Apache HBase &#8482; Reference Guide</h1>
<div class="details">
<span id="author" class="author">Apache HBase Team</span><br>
<span id="email" class="email">&lt;<a href="mailto:hbase-dev@lists.apache.org">hbase-dev@lists.apache.org</a>&gt;</span><br>
<span id="revnumber">version 3.0.0-SNAPSHOT</span>
</div>
<div id="toc" class="toc2">
<div id="toctitle">Contents</div>
<ul class="sectlevel1">
<li><a href="#_preface">Preface</a></li>
<li><a href="#getting_started">Getting Started</a>
<ul class="sectlevel1">
<li><a href="#_introduction">1. Introduction</a></li>
<li><a href="#quickstart">2. Quick Start - Standalone HBase</a></li>
</ul>
</li>
<li><a href="#configuration">Apache HBase Configuration</a>
<ul class="sectlevel1">
<li><a href="#_configuration_files">3. Configuration Files</a></li>
<li><a href="#basic.prerequisites">4. Basic Prerequisites</a></li>
<li><a href="#standalone_dist">5. HBase run modes: Standalone and Distributed</a></li>
<li><a href="#confirm">6. Running and Confirming Your Installation</a></li>
<li><a href="#config.files">7. Default Configuration</a></li>
<li><a href="#example_config">8. Example Configurations</a></li>
<li><a href="#important_configurations">9. The Important Configurations</a></li>
<li><a href="#dyn_config">10. Dynamic Configuration</a></li>
</ul>
</li>
<li><a href="#upgrading">Upgrading</a>
<ul class="sectlevel1">
<li><a href="#hbase.versioning">11. HBase version number and compatibility</a></li>
<li><a href="#_rollback">12. Rollback</a></li>
<li><a href="#_upgrade_paths">13. Upgrade Paths</a></li>
</ul>
</li>
<li><a href="#shell">The Apache HBase Shell</a>
<ul class="sectlevel1">
<li><a href="#scripting">14. Scripting with Ruby</a></li>
<li><a href="#_running_the_shell_in_non_interactive_mode">15. Running the Shell in Non-Interactive Mode</a></li>
<li><a href="#hbase.shell.noninteractive">16. HBase Shell in OS Scripts</a></li>
<li><a href="#_read_hbase_shell_commands_from_a_command_file">17. Read HBase Shell Commands from a Command File</a></li>
<li><a href="#_passing_vm_options_to_the_shell">18. Passing VM Options to the Shell</a></li>
<li><a href="#_overriding_configuration_starting_the_hbase_shell">19. Overriding configuration starting the HBase Shell</a></li>
<li><a href="#_shell_tricks">20. Shell Tricks</a></li>
</ul>
</li>
<li><a href="#datamodel">Data Model</a>
<ul class="sectlevel1">
<li><a href="#conceptual.view">21. Conceptual View</a></li>
<li><a href="#physical.view">22. Physical View</a></li>
<li><a href="#_namespace">23. Namespace</a></li>
<li><a href="#_table">24. Table</a></li>
<li><a href="#_row">25. Row</a></li>
<li><a href="#columnfamily">26. Column Family</a></li>
<li><a href="#_cells">27. Cells</a></li>
<li><a href="#_data_model_operations">28. Data Model Operations</a></li>
<li><a href="#versions">29. Versions</a></li>
<li><a href="#dm.sort">30. Sort Order</a></li>
<li><a href="#dm.column.metadata">31. Column Metadata</a></li>
<li><a href="#joins">32. Joins</a></li>
<li><a href="#_acid">33. ACID</a></li>
</ul>
</li>
<li><a href="#schema">HBase and Schema Design</a>
<ul class="sectlevel1">
<li><a href="#schema.creation">34. Schema Creation</a></li>
<li><a href="#table_schema_rules_of_thumb">35. Table Schema Rules Of Thumb</a></li>
</ul>
</li>
<li><a href="#regionserver_sizing_rules_of_thumb">RegionServer Sizing Rules of Thumb</a>
<ul class="sectlevel1">
<li><a href="#number.of.cfs">36. On the number of column families</a></li>
<li><a href="#rowkey.design">37. Rowkey Design</a></li>
<li><a href="#schema.versions">38. Number of Versions</a></li>
<li><a href="#supported.datatypes">39. Supported Datatypes</a></li>
<li><a href="#schema.joins">40. Joins</a></li>
<li><a href="#ttl">41. Time To Live (TTL)</a></li>
<li><a href="#cf.keep.deleted">42. Keeping Deleted Cells</a></li>
<li><a href="#secondary.indexes">43. Secondary Indexes and Alternate Query Paths</a></li>
<li><a href="#_constraints">44. Constraints</a></li>
<li><a href="#schema.casestudies">45. Schema Design Case Studies</a></li>
<li><a href="#schema.ops">46. Operational and Performance Configuration Options</a></li>
<li><a href="#_special_cases">47. Special Cases</a></li>
</ul>
</li>
<li><a href="#mapreduce">HBase and MapReduce</a>
<ul class="sectlevel1">
<li><a href="#hbase.mapreduce.classpath">48. HBase, MapReduce, and the CLASSPATH</a></li>
<li><a href="#_mapreduce_scan_caching">49. MapReduce Scan Caching</a></li>
<li><a href="#_bundled_hbase_mapreduce_jobs">50. Bundled HBase MapReduce Jobs</a></li>
<li><a href="#_hbase_as_a_mapreduce_job_data_source_and_data_sink">51. HBase as a MapReduce Job Data Source and Data Sink</a></li>
<li><a href="#_writing_hfiles_directly_during_bulk_import">52. Writing HFiles Directly During Bulk Import</a></li>
<li><a href="#_rowcounter_example">53. RowCounter Example</a></li>
<li><a href="#splitter">54. Map-Task Splitting</a></li>
<li><a href="#mapreduce.example">55. HBase MapReduce Examples</a></li>
<li><a href="#mapreduce.htable.access">56. Accessing Other HBase Tables in a MapReduce Job</a></li>
<li><a href="#mapreduce.specex">57. Speculative Execution</a></li>
<li><a href="#cascading">58. Cascading</a></li>
</ul>
</li>
<li><a href="#security">Securing Apache HBase</a>
<ul class="sectlevel1">
<li><a href="#_using_secure_http_https_for_the_web_ui">59. Using Secure HTTP (HTTPS) for the Web UI</a></li>
<li><a href="#hbase.secure.spnego.ui">60. Using SPNEGO for Kerberos authentication with Web UIs</a></li>
<li><a href="#hbase.secure.configuration">61. Secure Client Access to Apache HBase</a></li>
<li><a href="#hbase.secure.simpleconfiguration">62. Simple User Access to Apache HBase</a></li>
<li><a href="#_securing_access_to_hdfs_and_zookeeper">63. Securing Access to HDFS and ZooKeeper</a></li>
<li><a href="#_securing_access_to_your_data">64. Securing Access To Your Data</a></li>
<li><a href="#security.example.config">65. Security Configuration Example</a></li>
</ul>
</li>
<li><a href="#_architecture">Architecture</a>
<ul class="sectlevel1">
<li><a href="#arch.overview">66. Overview</a></li>
<li><a href="#arch.catalog">67. Catalog Tables</a></li>
<li><a href="#architecture.client">68. Client</a></li>
<li><a href="#client.filter">69. Client Request Filters</a></li>
<li><a href="#architecture.master">70. Master</a></li>
<li><a href="#regionserver.arch">71. RegionServer</a></li>
<li><a href="#regions.arch">72. Regions</a></li>
<li><a href="#arch.bulk.load">73. Bulk Loading</a></li>
<li><a href="#arch.hdfs">74. HDFS</a></li>
<li><a href="#arch.timelineconsistent.reads">75. Timeline-consistent High Available Reads</a></li>
<li><a href="#hbase_mob">76. Storing Medium-sized Objects (MOB)</a></li>
</ul>
</li>
<li><a href="#inmemory_compaction">In-memory Compaction</a>
<ul class="sectlevel1">
<li><a href="#imc.overview">77. Overview</a></li>
<li><a href="#_enabling">78. Enabling</a></li>
</ul>
</li>
<li><a href="#offheap_read_write">RegionServer Offheap Read/Write Path</a>
<ul class="sectlevel1">
<li><a href="#regionserver.offheap.overview">79. Overview</a></li>
<li><a href="#regionserver.offheap.readpath">80. Offheap read-path</a></li>
<li><a href="#regionserver.read.hdfs.block.offheap">81. Read block from HDFS to offheap directly</a></li>
<li><a href="#regionserver.offheap.writepath">82. Offheap write-path</a></li>
</ul>
</li>
<li><a href="#backuprestore">Backup and Restore</a>
<ul class="sectlevel1">
<li><a href="#br.overview">83. Overview</a></li>
<li><a href="#br.terminology">84. Terminology</a></li>
<li><a href="#br.planning">85. Planning</a></li>
<li><a href="#br.initial.setup">86. First-time configuration steps</a></li>
<li><a href="#_backup_and_restore_commands">87. Backup and Restore commands</a></li>
<li><a href="#br.administration">88. Administration of Backup Images</a></li>
<li><a href="#br.backup.configuration">89. Configuration keys</a></li>
<li><a href="#br.best.practices">90. Best Practices</a></li>
<li><a href="#br.s3.backup.scenario">91. Scenario: Safeguarding Application Datasets on Amazon S3</a></li>
<li><a href="#br.data.security">92. Security of Backup Data</a></li>
<li><a href="#br.technical.details">93. Technical Details of Incremental Backup and Restore</a></li>
<li><a href="#br.filesystem.growth.warning">94. A Warning on File System Growth</a></li>
<li><a href="#br.backup.capacity.planning">95. Capacity Planning</a></li>
<li><a href="#br.limitations">96. Limitations of the Backup and Restore Utility</a></li>
</ul>
</li>
<li><a href="#syncreplication">Synchronous Replication</a>
<ul class="sectlevel1">
<li><a href="#_background">97. Background</a></li>
<li><a href="#_design">98. Design</a></li>
<li><a href="#_operation_and_maintenance">99. Operation and maintenance</a></li>
</ul>
</li>
<li><a href="#hbase_apis">Apache HBase APIs</a>
<ul class="sectlevel1">
<li><a href="#_examples">100. Examples</a></li>
</ul>
</li>
<li><a href="#external_apis">Apache HBase External APIs</a>
<ul class="sectlevel1">
<li><a href="#_rest">101. REST</a></li>
<li><a href="#_thrift">102. Thrift</a></li>
<li><a href="#c">103. C/C++ Apache HBase Client</a></li>
<li><a href="#jdo">104. Using Java Data Objects (JDO) with HBase</a></li>
<li><a href="#scala">105. Scala</a></li>
<li><a href="#jython">106. Jython</a></li>
</ul>
</li>
<li><a href="#thrift">Thrift API and Filter Language</a>
<ul class="sectlevel1">
<li><a href="#thrift.filter_language">107. Filter Language</a></li>
</ul>
</li>
<li><a href="#spark">HBase and Spark</a>
<ul class="sectlevel1">
<li><a href="#_basic_spark">108. Basic Spark</a></li>
<li><a href="#_spark_streaming">109. Spark Streaming</a></li>
<li><a href="#_bulk_load">110. Bulk Load</a></li>
<li><a href="#_sparksql_dataframes">111. SparkSQL/DataFrames</a></li>
</ul>
</li>
<li><a href="#cp">Apache HBase Coprocessors</a>
<ul class="sectlevel1">
<li><a href="#_coprocessor_overview">112. Coprocessor Overview</a></li>
<li><a href="#_types_of_coprocessors">113. Types of Coprocessors</a></li>
<li><a href="#cp_loading">114. Loading Coprocessors</a></li>
<li><a href="#cp_example">115. Examples</a></li>
<li><a href="#_guidelines_for_deploying_a_coprocessor">116. Guidelines For Deploying A Coprocessor</a></li>
<li><a href="#_restricting_coprocessor_usage">117. Restricting Coprocessor Usage</a></li>
</ul>
</li>
<li><a href="#performance">Apache HBase Performance Tuning</a>
<ul class="sectlevel1">
<li><a href="#perf.os">118. Operating System</a></li>
<li><a href="#perf.network">119. Network</a></li>
<li><a href="#jvm">120. Java</a></li>
<li><a href="#perf.configurations">121. HBase Configurations</a></li>
<li><a href="#perf.zookeeper">122. ZooKeeper</a></li>
<li><a href="#perf.schema">123. Schema Design</a></li>
<li><a href="#perf.general">124. HBase General Patterns</a></li>
<li><a href="#perf.writing">125. Writing to HBase</a></li>
<li><a href="#perf.reading">126. Reading from HBase</a></li>
<li><a href="#perf.deleting">127. Deleting from HBase</a></li>
<li><a href="#perf.hdfs">128. HDFS</a></li>
<li><a href="#perf.ec2">129. Amazon EC2</a></li>
<li><a href="#perf.hbase.mr.cluster">130. Collocating HBase and MapReduce</a></li>
<li><a href="#perf.casestudy">131. Case Studies</a></li>
</ul>
</li>
<li><a href="#profiler">Profiler Servlet</a>
<ul class="sectlevel1">
<li><a href="#_background_2">132. Background</a></li>
<li><a href="#_prerequisites_2">133. Prerequisites</a></li>
<li><a href="#_usage">134. Usage</a></li>
<li><a href="#_ui">135. UI</a></li>
<li><a href="#_notes">136. Notes</a></li>
</ul>
</li>
<li><a href="#trouble">Troubleshooting and Debugging Apache HBase</a>
<ul class="sectlevel1">
<li><a href="#trouble.general">137. General Guidelines</a></li>
<li><a href="#trouble.log">138. Logs</a></li>
<li><a href="#trouble.resources">139. Resources</a></li>
<li><a href="#trouble.tools">140. Tools</a></li>
<li><a href="#trouble.client">141. Client</a></li>
<li><a href="#trouble.mapreduce">142. MapReduce</a></li>
<li><a href="#trouble.namenode">143. NameNode</a></li>
<li><a href="#trouble.network">144. Network</a></li>
<li><a href="#trouble.rs">145. RegionServer</a></li>
<li><a href="#trouble.master">146. Master</a></li>
<li><a href="#trouble.zookeeper">147. ZooKeeper</a></li>
<li><a href="#trouble.ec2">148. Amazon EC2</a></li>
<li><a href="#trouble.versions">149. HBase and Hadoop version issues</a></li>
<li><a href="#_hbase_and_hdfs">150. HBase and HDFS</a></li>
<li><a href="#trouble.tests">151. Running unit or integration tests</a></li>
<li><a href="#trouble.casestudy">152. Case Studies</a></li>
<li><a href="#trouble.crypto">153. Cryptographic Features</a></li>
<li><a href="#_operating_system_specific_issues">154. Operating System Specific Issues</a></li>
<li><a href="#_jdk_issues">155. JDK Issues</a></li>
</ul>
</li>
<li><a href="#casestudies">Apache HBase Case Studies</a>
<ul class="sectlevel1">
<li><a href="#casestudies.overview">156. Overview</a></li>
<li><a href="#casestudies.schema">157. Schema Design</a></li>
<li><a href="#casestudies.perftroub">158. Performance/Troubleshooting</a></li>
</ul>
</li>
<li><a href="#ops_mgt">Apache HBase Operational Management</a>
<ul class="sectlevel1">
<li><a href="#tools">159. HBase Tools and Utilities</a></li>
<li><a href="#ops.regionmgt">160. Region Management</a></li>
<li><a href="#node.management">161. Node Management</a></li>
<li><a href="#hbase_metrics">162. HBase Metrics</a></li>
<li><a href="#ops.monitoring">163. HBase Monitoring</a></li>
<li><a href="#_cluster_replication">164. Cluster Replication</a></li>
<li><a href="#_running_multiple_workloads_on_a_single_cluster">165. Running Multiple Workloads On a Single Cluster</a></li>
<li><a href="#ops.backup">166. HBase Backup</a></li>
<li><a href="#ops.snapshots">167. HBase Snapshots</a></li>
<li><a href="#snapshots_azure">168. Storing Snapshots in Microsoft Azure Blob Storage</a></li>
<li><a href="#ops.capacity">169. Capacity Planning and Region Sizing</a></li>
<li><a href="#table.rename">170. Table Rename</a></li>
<li><a href="#rsgroup">171. RegionServer Grouping</a></li>
<li><a href="#normalizer">172. Region Normalizer</a></li>
</ul>
</li>
<li><a href="#developer">Building and Developing Apache HBase</a>
<ul class="sectlevel1">
<li><a href="#getting.involved">173. Getting Involved</a></li>
<li><a href="#repos">174. Apache HBase Repositories</a></li>
<li><a href="#_ides">175. IDEs</a></li>
<li><a href="#build">176. Building Apache HBase</a></li>
<li><a href="#releasing">177. Releasing Apache HBase</a></li>
<li><a href="#hbase.rc.voting">178. Voting on Release Candidates</a></li>
<li><a href="#hbase.release.announcement">179. Announcing Releases</a></li>
<li><a href="#documentation">180. Generating the HBase Reference Guide</a></li>
<li><a href="#hbase.org">181. Updating <a href="https://hbase.apache.org">hbase.apache.org</a></a></li>
<li><a href="#hbase.tests">182. Tests</a></li>
<li><a href="#developing">183. Developer Guidelines</a></li>
</ul>
</li>
<li><a href="#unit.tests">Unit Testing HBase Applications</a>
<ul class="sectlevel1">
<li><a href="#_junit">184. JUnit</a></li>
<li><a href="#mockito">185. Mockito</a></li>
<li><a href="#_mrunit">186. MRUnit</a></li>
<li><a href="#_integration_testing_with_an_hbase_mini_cluster">187. Integration Testing with an HBase Mini-Cluster</a></li>
</ul>
</li>
<li><a href="#protobuf">Protobuf in HBase</a>
<ul class="sectlevel1">
<li><a href="#_protobuf">188. Protobuf</a></li>
</ul>
</li>
<li><a href="#pv2">Procedure Framework (Pv2): <a href="https://issues.apache.org/jira/browse/HBASE-12439">HBASE-12439</a></a>
<ul class="sectlevel1">
<li><a href="#_procedures">189. Procedures</a></li>
<li><a href="#_subprocedures">190. Subprocedures</a></li>
<li><a href="#_procedureexecutor">191. ProcedureExecutor</a></li>
<li><a href="#_nonces">192. Nonces</a></li>
<li><a href="#_wait_wake_suspend_yield">193. Wait/Wake/Suspend/Yield</a></li>
<li><a href="#_locking">194. Locking</a></li>
<li><a href="#_procedure_types">195. Procedure Types</a></li>
<li><a href="#_references">196. References</a></li>
</ul>
</li>
<li><a href="#amv2">AMv2 Description for Devs</a>
<ul class="sectlevel1">
<li><a href="#_background_3">197. Background</a></li>
<li><a href="#_new_system">198. New System</a></li>
<li><a href="#_procedures_detail">199. Procedures Detail</a></li>
<li><a href="#_ui_2">200. UI</a></li>
<li><a href="#_logging">201. Logging</a></li>
<li><a href="#_implementation_notes">202. Implementation Notes</a></li>
<li><a href="#_new_configs">203. New Configs</a></li>
<li><a href="#_tools">204. Tools</a></li>
</ul>
</li>
<li><a href="#zookeeper">ZooKeeper</a>
<ul class="sectlevel1">
<li><a href="#_using_existing_zookeeper_ensemble">205. Using existing ZooKeeper ensemble</a></li>
<li><a href="#zk.sasl.auth">206. SASL Authentication with ZooKeeper</a></li>
</ul>
</li>
<li><a href="#community">Community</a>
<ul class="sectlevel1">
<li><a href="#_decisions">207. Decisions</a></li>
<li><a href="#community.roles">208. Community Roles</a></li>
<li><a href="#hbase.commit.msg.format">209. Commit Message format</a></li>
</ul>
</li>
<li><a href="#_appendix">Appendix</a>
<ul class="sectlevel1">
<li><a href="#appendix_contributing_to_documentation">Appendix A: Contributing to Documentation</a></li>
<li><a href="#faq">Appendix B: FAQ</a></li>
<li><a href="#appendix_acl_matrix">Appendix C: Access Control Matrix</a></li>
<li><a href="#compression">Appendix D: Compression and Data Block Encoding In HBase</a></li>
<li><a href="#sql">Appendix E: SQL over HBase</a></li>
<li><a href="#ycsb">Appendix F: YCSB</a></li>
<li><a href="#_hfile_format_2">Appendix G: HFile format</a></li>
<li><a href="#other.info">Appendix H: Other Information About HBase</a></li>
<li><a href="#hbase.history">Appendix I: HBase History</a></li>
<li><a href="#asf">Appendix J: HBase and the Apache Software Foundation</a></li>
<li><a href="#orca">Appendix K: Apache HBase Orca</a></li>
<li><a href="#tracing">Appendix L: Enabling Dapper-like Tracing in HBase</a></li>
<li><a href="#tracing.client.modifications">210. Client Modifications</a></li>
<li><a href="#tracing.client.shell">211. Tracing from HBase Shell</a></li>
<li><a href="#hbase.rpc">Appendix M: 0.95 RPC Specification</a></li>
<li><a href="#_known_incompatibilities_among_hbase_versions">Appendix N: Known Incompatibilities Among HBase Versions</a></li>
<li><a href="#_hbase_2_0_incompatible_changes">212. HBase 2.0 Incompatible Changes</a></li>
</ul>
</li>
</ul>
</div>
</div>
<div id="content">
<div id="preamble">
<div class="sectionbody">
<div>
<a href="https://hbase.apache.org"><img src="images/hbase_logo_with_orca.png" alt="Apache HBase Logo" /></a>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_preface"><a class="anchor" href="#_preface"></a>Preface</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This is the official reference guide for the <a href="https://hbase.apache.org/">HBase</a> version it ships with.</p>
</div>
<div class="paragraph">
<p>Herein you will find either the definitive documentation on an HBase topic as of its
standing when the referenced HBase version shipped, or it will point to the location
in <a href="https://hbase.apache.org/apidocs/index.html">Javadoc</a> or
<a href="https://issues.apache.org/jira/browse/HBASE">JIRA</a> where the pertinent information can be found.</p>
</div>
<div class="paragraph">
<div class="title">About This Guide</div>
<p>This reference guide is a work in progress. The source for this guide can be found in the
_src/main/asciidoc directory of the HBase source. This reference guide is marked up
using <a href="http://asciidoc.org/">AsciiDoc</a> from which the finished guide is generated as part of the
'site' build target. Run</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn site</code></pre>
</div>
</div>
<div class="paragraph">
<p>to generate this documentation.
Amendments and improvements to the documentation are welcomed.
Click
<a href="https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12310753&amp;issuetype=1&amp;components=12312132&amp;summary=SHORT+DESCRIPTION">this link</a>
to file a new documentation bug against Apache HBase with some values pre-selected.</p>
</div>
<div class="paragraph">
<div class="title">Contributing to the Documentation</div>
<p>For an overview of AsciiDoc and suggestions to get started contributing to the documentation,
see the <a href="#appendix_contributing_to_documentation">relevant section later in this documentation</a>.</p>
</div>
<div class="paragraph">
<div class="title">Heads-up if this is your first foray into the world of distributed computing&#8230;&#8203;</div>
<p>If this is your first foray into the wonderful world of Distributed Computing, then you are in for some interesting times.
First off, distributed systems are hard; making a distributed system hum requires a disparate skillset that spans systems (hardware and software) and networking.</p>
</div>
<div class="paragraph">
<p>Your cluster&#8217;s operation can hiccup because of any of a myriad set of reasons from bugs in HBase itself through misconfigurations&#8201;&#8212;&#8201;misconfiguration of HBase but also operating system misconfigurations&#8201;&#8212;&#8201;through to hardware problems whether it be a bug in your network card drivers or an underprovisioned RAM bus (to mention two recent examples of hardware issues that manifested as "HBase is slow"). You will also need to do a recalibration if up to this your computing has been bound to a single box.
Here is one good starting point: <a href="http://en.wikipedia.org/wiki/Fallacies_of_Distributed_Computing">Fallacies of Distributed Computing</a>.</p>
</div>
<div class="paragraph">
<p>That said, you are welcome.<br>
It&#8217;s a fun place to be.<br>
Yours, the HBase Community.</p>
</div>
<div class="paragraph">
<div class="title">Reporting Bugs</div>
<p>Please use <a href="https://issues.apache.org/jira/browse/hbase">JIRA</a> to report non-security-related bugs.</p>
</div>
<div class="paragraph">
<p>To protect existing HBase installations from new vulnerabilities, please <strong>do not</strong> use JIRA to report security-related bugs. Instead, send your report to the mailing list <a href="mailto:private@hbase.apache.org">private@hbase.apache.org</a>, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.</p>
</div>
<div id="hbase_supported_tested_definitions" class="paragraph">
<div class="title">Support and Testing Expectations</div>
<p>The phrases /supported/, /not supported/, /tested/, and /not tested/ occur several
places throughout this guide. In the interest of clarity, here is a brief explanation
of what is generally meant by these phrases, in the context of HBase.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Commercial technical support for Apache HBase is provided by many Hadoop vendors.
This is not the sense in which the term /support/ is used in the context of the
Apache HBase project. The Apache HBase team assumes no responsibility for your
HBase clusters, your configuration, or your data.
</td>
</tr>
</table>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Supported</dt>
<dd>
<p>In the context of Apache HBase, /supported/ means that HBase is designed to work
in the way described, and deviation from the defined behavior or functionality should
be reported as a bug.</p>
</dd>
<dt class="hdlist1">Not Supported</dt>
<dd>
<p>In the context of Apache HBase, /not supported/ means that a use case or use pattern
is not expected to work and should be considered an antipattern. If you think this
designation should be reconsidered for a given feature or use pattern, file a JIRA
or start a discussion on one of the mailing lists.</p>
</dd>
<dt class="hdlist1">Tested</dt>
<dd>
<p>In the context of Apache HBase, /tested/ means that a feature is covered by unit
or integration tests, and has been proven to work as expected.</p>
</dd>
<dt class="hdlist1">Not Tested</dt>
<dd>
<p>In the context of Apache HBase, /not tested/ means that a feature or use pattern
may or may not work in a given way, and may or may not corrupt your data or cause
operational issues. It is an unknown, and there are no guarantees. If you can provide
proof that a feature designated as /not tested/ does work in a given way, please
submit the tests and/or the metrics so that other users can gain certainty about
such features or use patterns.</p>
</dd>
</dl>
</div>
</div>
</div>
<h1 id="getting_started" class="sect0"><a class="anchor" href="#getting_started"></a>Getting Started</h1>
<div class="sect1">
<h2 id="_introduction"><a class="anchor" href="#_introduction"></a>1. Introduction</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="#quickstart">Quickstart</a> will get you up and running on a single-node, standalone instance of HBase.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="quickstart"><a class="anchor" href="#quickstart"></a>2. Quick Start - Standalone HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This section describes the setup of a single-node standalone HBase.
A <em>standalone</em> instance has all HBase daemons&#8201;&#8212;&#8201;the Master, RegionServers,
and ZooKeeper&#8201;&#8212;&#8201;running in a single JVM persisting to the local filesystem.
It is our most basic deploy profile. We will show you how
to create a table in HBase using the <code>hbase shell</code> CLI,
insert rows into the table, perform put and scan operations against the
table, enable or disable the table, and start and stop HBase.</p>
</div>
<div class="paragraph">
<p>Apart from downloading HBase, this procedure should take less than 10 minutes.</p>
</div>
<div class="sect2">
<h3 id="_jdk_version_requirements"><a class="anchor" href="#_jdk_version_requirements"></a>2.1. JDK Version Requirements</h3>
<div class="paragraph">
<p>HBase requires that a JDK be installed.
See <a href="#java">Java</a> for information about supported JDK versions.</p>
</div>
</div>
<div class="sect2">
<h3 id="_get_started_with_hbase"><a class="anchor" href="#_get_started_with_hbase"></a>2.2. Get Started with HBase</h3>
<div class="olist arabic">
<div class="title">Procedure: Download, Configure, and Start HBase in Standalone Mode</div>
<ol class="arabic">
<li>
<p>Choose a download site from this list of <a href="https://www.apache.org/dyn/closer.lua/hbase/">Apache Download Mirrors</a>.
Click on the suggested top link.
This will take you to a mirror of <em>HBase Releases</em>.
Click on the folder named <em>stable</em> and then download the binary file that ends in <em>.tar.gz</em> to your local filesystem.
Do not download the file ending in <em>src.tar.gz</em> for now.</p>
</li>
<li>
<p>Extract the downloaded file, and change to the newly-created directory.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">$ tar xzvf hbase-3.0.0-SNAPSHOT-bin.tar.gz
$ cd hbase-3.0.0-SNAPSHOT/</code></pre>
</div>
</div>
</li>
<li>
<p>You must set the <code>JAVA_HOME</code> environment variable before starting HBase.
To make this easier, HBase lets you set it within the <em>conf/hbase-env.sh</em> file. You must locate where Java is
installed on your machine, and one way to find this is by using the <em>whereis java</em> command. Once you have the location,
edit the <em>conf/hbase-env.sh</em> file and uncomment the line starting with <em>#export JAVA_HOME=</em>, and then set it to your Java installation path.</p>
<div class="literalblock">
<div class="title">Example extract from <em>hbase-env.sh</em> where <em>JAVA_HOME</em> is set</div>
<div class="content">
<pre># Set environment variables here.
# The java implementation to use.
export JAVA_HOME=/usr/jdk64/jdk1.8.0_112</pre>
</div>
</div>
</li>
<li>
<p>Edit <em>conf/hbase-site.xml</em>, which is the main HBase configuration file.
At this time, you need to specify the directory on the local filesystem where HBase and ZooKeeper write data and acknowledge some risks.
By default, a new directory is created under /tmp.
Many servers are configured to delete the contents of <em>/tmp</em> upon reboot, so you should store the data elsewhere.
The following configuration will store HBase&#8217;s data in the <em>hbase</em> directory, in the home directory of the user called <code>testuser</code>.
Paste the <code>&lt;property&gt;</code> tags beneath the <code>&lt;configuration&gt;</code> tags, which should be empty in a new HBase install.</p>
<div class="exampleblock">
<div class="title">Example 1. Example <em>hbase-site.xml</em> for Standalone HBase</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rootdir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>file:///home/testuser/hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.dataDir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/home/testuser/zookeeper<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.unsafe.stream.capability.enforce<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>false<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Controls whether HBase will check for stream capabilities (hflush/hsync).
Disable this if you intend to run on LocalFileSystem, denoted by a rootdir
with the 'file://' scheme, but be mindful of the NOTE below.
WARNING: Setting this to false blinds you to potential data loss and
inconsistent system state in the event of process and/or node failures. If
HBase is complaining of an inability to use hsync or hflush it's most
likely not a false positive.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>You do not need to create the HBase data directory.
HBase will do this for you. If you create the directory,
HBase will attempt to do a migration, which is not what you want.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The <em>hbase.rootdir</em> in the above example points to a directory
in the <em>local filesystem</em>. The 'file://' prefix is how we denote local
filesystem. You should take the WARNING present in the configuration example
to heart. In standalone mode HBase makes use of the local filesystem abstraction
from the Apache Hadoop project. That abstraction doesn&#8217;t provide the durability
promises that HBase needs to operate safely. This is fine for local development
and testing use cases where the cost of cluster failure is well contained. It is
not appropriate for production deployments; eventually you will lose data.
</td>
</tr>
</table>
</div>
</li>
</ol>
</div>
<div class="paragraph">
<p>To home HBase on an existing instance of HDFS, set the <em>hbase.rootdir</em> to point at a
directory up on your instance: e.g. <em>hdfs://namenode.example.org:8020/hbase</em>.
For more on this variant, see the section below on Standalone HBase over HDFS.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The <em>bin/start-hbase.sh</em> script is provided as a convenient way to start HBase.
Issue the command, and if all goes well, a message is logged to standard output showing that HBase started successfully.
You can use the <code>jps</code> command to verify that you have one running process called <code>HMaster</code>.
In standalone mode HBase runs all daemons within this single JVM, i.e.
the HMaster, a single HRegionServer, and the ZooKeeper daemon.
Go to <em><a href="http://localhost:16010" class="bare">http://localhost:16010</a></em> to view the HBase Web UI.</p>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Java needs to be installed and available.
If you get an error indicating that Java is not installed,
but it is on your system, perhaps in a non-standard location,
edit the <em>conf/hbase-env.sh</em> file and modify the <code>JAVA_HOME</code>
setting to point to the directory that contains <em>bin/java</em> on your system.
</td>
</tr>
</table>
</div>
</li>
</ol>
</div>
<div id="shell_exercises" class="olist arabic">
<div class="title">Procedure: Use HBase For the First Time</div>
<ol class="arabic">
<li>
<p>Connect to HBase.</p>
<div class="paragraph">
<p>Connect to your running instance of HBase using the <code>hbase shell</code> command, located in the <em class="path">bin/</em> directory of your HBase install.
In this example, some usage and version information that is printed when you start HBase Shell has been omitted.
The HBase Shell prompt ends with a <code>&gt;</code> character.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase(main):001:0&gt;</pre>
</div>
</div>
</li>
<li>
<p>Display HBase Shell Help Text.</p>
<div class="paragraph">
<p>Type <code>help</code> and press Enter, to display some basic usage information for HBase Shell, as well as several example commands.
Notice that table names, rows, columns all must be enclosed in quote characters.</p>
</div>
</li>
<li>
<p>Create a table.</p>
<div class="paragraph">
<p>Use the <code>create</code> command to create a new table.
You must specify the table name and the ColumnFamily name.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):001:0&gt; create 'test', 'cf'
0 row(s) in 0.4170 seconds
=&gt; Hbase::Table - test</pre>
</div>
</div>
</li>
<li>
<p>List Information About your Table</p>
<div class="paragraph">
<p>Use the <code>list</code> command to confirm your table exists</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):002:0&gt; list 'test'
TABLE
test
1 row(s) in 0.0180 seconds
=&gt; ["test"]</pre>
</div>
</div>
<div class="paragraph">
<p>Now use the <code>describe</code> command to see details, including configuration defaults</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):003:0&gt; describe 'test'
Table test is ENABLED
test
COLUMN FAMILIES DESCRIPTION
{NAME =&gt; 'cf', VERSIONS =&gt; '1', EVICT_BLOCKS_ON_CLOSE =&gt; 'false', NEW_VERSION_BEHAVIOR =&gt; 'false', KEEP_DELETED_CELLS =&gt; 'FALSE', CACHE_DATA_ON_WRITE =&gt;
'false', DATA_BLOCK_ENCODING =&gt; 'NONE', TTL =&gt; 'FOREVER', MIN_VERSIONS =&gt; '0', REPLICATION_SCOPE =&gt; '0', BLOOMFILTER =&gt; 'ROW', CACHE_INDEX_ON_WRITE =&gt; 'f
alse', IN_MEMORY =&gt; 'false', CACHE_BLOOMS_ON_WRITE =&gt; 'false', PREFETCH_BLOCKS_ON_OPEN =&gt; 'false', COMPRESSION =&gt; 'NONE', BLOCKCACHE =&gt; 'true', BLOCKSIZE
=&gt; '65536'}
1 row(s)
Took 0.9998 seconds</pre>
</div>
</div>
</li>
<li>
<p>Put data into your table.</p>
<div class="paragraph">
<p>To put data into your table, use the <code>put</code> command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):003:0&gt; put 'test', 'row1', 'cf:a', 'value1'
0 row(s) in 0.0850 seconds
hbase(main):004:0&gt; put 'test', 'row2', 'cf:b', 'value2'
0 row(s) in 0.0110 seconds
hbase(main):005:0&gt; put 'test', 'row3', 'cf:c', 'value3'
0 row(s) in 0.0100 seconds</pre>
</div>
</div>
<div class="paragraph">
<p>Here, we insert three values, one at a time.
The first insert is at <code>row1</code>, column <code>cf:a</code>, with a value of <code>value1</code>.
Columns in HBase are comprised of a column family prefix, <code>cf</code> in this example, followed by a colon and then a column qualifier suffix, <code>a</code> in this case.</p>
</div>
</li>
<li>
<p>Scan the table for all data at once.</p>
<div class="paragraph">
<p>One of the ways to get data from HBase is to scan.
Use the <code>scan</code> command to scan the table for data.
You can limit your scan, but for now, all data is fetched.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):006:0&gt; scan 'test'
ROW COLUMN+CELL
row1 column=cf:a, timestamp=1421762485768, value=value1
row2 column=cf:b, timestamp=1421762491785, value=value2
row3 column=cf:c, timestamp=1421762496210, value=value3
3 row(s) in 0.0230 seconds</pre>
</div>
</div>
</li>
<li>
<p>Get a single row of data.</p>
<div class="paragraph">
<p>To get a single row of data at a time, use the <code>get</code> command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):007:0&gt; get 'test', 'row1'
COLUMN CELL
cf:a timestamp=1421762485768, value=value1
1 row(s) in 0.0350 seconds</pre>
</div>
</div>
</li>
<li>
<p>Disable a table.</p>
<div class="paragraph">
<p>If you want to delete a table or change its settings, as well as in some other situations, you need to disable the table first, using the <code>disable</code> command.
You can re-enable it using the <code>enable</code> command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):008:0&gt; disable 'test'
0 row(s) in 1.1820 seconds
hbase(main):009:0&gt; enable 'test'
0 row(s) in 0.1770 seconds</pre>
</div>
</div>
<div class="paragraph">
<p>Disable the table again if you tested the <code>enable</code> command above:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):010:0&gt; disable 'test'
0 row(s) in 1.1820 seconds</pre>
</div>
</div>
</li>
<li>
<p>Drop the table.</p>
<div class="paragraph">
<p>To drop (delete) a table, use the <code>drop</code> command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):011:0&gt; drop 'test'
0 row(s) in 0.1370 seconds</pre>
</div>
</div>
</li>
<li>
<p>Exit the HBase Shell.</p>
<div class="paragraph">
<p>To exit the HBase Shell and disconnect from your cluster, use the <code>quit</code> command.
HBase is still running in the background.</p>
</div>
</li>
</ol>
</div>
<div class="olist arabic">
<div class="title">Procedure: Stop HBase</div>
<ol class="arabic">
<li>
<p>In the same way that the <em>bin/start-hbase.sh</em> script is provided to conveniently start all HBase daemons, the <em>bin/stop-hbase.sh</em> script stops them.</p>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/stop-hbase.sh
stopping hbase....................
$</pre>
</div>
</div>
</li>
<li>
<p>After issuing the command, it can take several minutes for the processes to shut down.
Use the <code>jps</code> to be sure that the HMaster and HRegionServer processes are shut down.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The above has shown you how to start and stop a standalone instance of HBase.
In the next sections we give a quick overview of other modes of hbase deploy.</p>
</div>
</div>
<div class="sect2">
<h3 id="quickstart_pseudo"><a class="anchor" href="#quickstart_pseudo"></a>2.3. Pseudo-Distributed Local Install</h3>
<div class="paragraph">
<p>After working your way through <a href="#quickstart">quickstart</a> standalone mode,
you can re-configure HBase to run in pseudo-distributed mode.
Pseudo-distributed mode means that HBase still runs completely on a single host,
but each HBase daemon (HMaster, HRegionServer, and ZooKeeper) runs as a separate process:
in standalone mode all daemons ran in one jvm process/instance.
By default, unless you configure the <code>hbase.rootdir</code> property as described in
<a href="#quickstart">quickstart</a>, your data is still stored in <em>/tmp/</em>.
In this walk-through, we store your data in HDFS instead, assuming you have HDFS available.
You can skip the HDFS configuration to continue storing your data in the local filesystem.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Hadoop Configuration</div>
<div class="paragraph">
<p>This procedure assumes that you have configured Hadoop and HDFS on your local system and/or a remote
system, and that they are running and available. It also assumes you are using Hadoop 2.
The guide on
<a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/SingleCluster.html">Setting up a Single Node Cluster</a>
in the Hadoop documentation is a good starting point.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Stop HBase if it is running.</p>
<div class="paragraph">
<p>If you have just finished <a href="#quickstart">quickstart</a> and HBase is still running, stop it.
This procedure will create a totally new directory where HBase will store its data, so any databases you created before will be lost.</p>
</div>
</li>
<li>
<p>Configure HBase.</p>
<div class="paragraph">
<p>Edit the <em>hbase-site.xml</em> configuration.
First, add the following property which directs HBase to run in distributed mode, with one JVM instance per daemon.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.cluster.distributed<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Next, change the <code>hbase.rootdir</code> from the local filesystem to the address of your HDFS instance, using the <code>hdfs:////</code> URI syntax.
In this example, HDFS is running on the localhost at port 8020. Be sure to either remove the entry for <code>hbase.unsafe.stream.capability.enforce</code> or set it to true.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rootdir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hdfs://localhost:8020/hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>You do not need to create the directory in HDFS.
HBase will do this for you.
If you create the directory, HBase will attempt to do a migration, which is not what you want.</p>
</div>
</li>
<li>
<p>Start HBase.</p>
<div class="paragraph">
<p>Use the <em>bin/start-hbase.sh</em> command to start HBase.
If your system is configured correctly, the <code>jps</code> command should show the HMaster and HRegionServer processes running.</p>
</div>
</li>
<li>
<p>Check the HBase directory in HDFS.</p>
<div class="paragraph">
<p>If everything worked correctly, HBase created its directory in HDFS.
In the configuration above, it is stored in <em>/hbase/</em> on HDFS.
You can use the <code>hadoop fs</code> command in Hadoop&#8217;s <em>bin/</em> directory to list this directory.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hadoop fs -ls /hbase
Found 7 items
drwxr-xr-x - hbase users 0 2014-06-25 18:58 /hbase/.tmp
drwxr-xr-x - hbase users 0 2014-06-25 21:49 /hbase/WALs
drwxr-xr-x - hbase users 0 2014-06-25 18:48 /hbase/corrupt
drwxr-xr-x - hbase users 0 2014-06-25 18:58 /hbase/data
-rw-r--r-- 3 hbase users 42 2014-06-25 18:41 /hbase/hbase.id
-rw-r--r-- 3 hbase users 7 2014-06-25 18:41 /hbase/hbase.version
drwxr-xr-x - hbase users 0 2014-06-25 21:49 /hbase/oldWALs</pre>
</div>
</div>
</li>
<li>
<p>Create a table and populate it with data.</p>
<div class="paragraph">
<p>You can use the HBase Shell to create a table, populate it with data, scan and get values from it, using the same procedure as in <a href="#shell_exercises">shell exercises</a>.</p>
</div>
</li>
<li>
<p>Start and stop a backup HBase Master (HMaster) server.</p>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Running multiple HMaster instances on the same hardware does not make sense in a production environment, in the same way that running a pseudo-distributed cluster does not make sense for production.
This step is offered for testing and learning purposes only.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The HMaster server controls the HBase cluster.
You can start up to 9 backup HMaster servers, which makes 10 total HMasters, counting the primary.
To start a backup HMaster, use the <code>local-master-backup.sh</code>.
For each backup master you want to start, add a parameter representing the port offset for that master.
Each HMaster uses two ports (16000 and 16010 by default). The port offset is added to these ports, so using an offset of 2, the backup HMaster would use ports 16002 and 16012.
The following command starts 3 backup servers using ports 16002/16012, 16003/16013, and 16005/16015.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/local-master-backup.sh start 2 3 5</pre>
</div>
</div>
<div class="paragraph">
<p>To kill a backup master without killing the entire cluster, you need to find its process ID (PID). The PID is stored in a file with a name like <em>/tmp/hbase-USER-X-master.pid</em>.
The only contents of the file is the PID.
You can use the <code>kill -9</code> command to kill that PID.
The following command will kill the master with port offset 1, but leave the cluster running:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ cat /tmp/hbase-testuser-1-master.pid |xargs kill -9</pre>
</div>
</div>
</li>
<li>
<p>Start and stop additional RegionServers</p>
<div class="paragraph">
<p>The HRegionServer manages the data in its StoreFiles as directed by the HMaster.
Generally, one HRegionServer runs per node in the cluster.
Running multiple HRegionServers on the same system can be useful for testing in pseudo-distributed mode.
The <code>local-regionservers.sh</code> command allows you to run multiple RegionServers.
It works in a similar way to the <code>local-master-backup.sh</code> command, in that each parameter you provide represents the port offset for an instance.
Each RegionServer requires two ports, and the default ports are 16020 and 16030.
Since HBase version 1.1.0, HMaster doesn&#8217;t use region server ports, this leaves 10 ports (16020 to 16029 and 16030 to 16039) to be used for RegionServers.
For supporting additional RegionServers, set environment variables HBASE_RS_BASE_PORT and HBASE_RS_INFO_BASE_PORT to appropriate values before running script <code>local-regionservers.sh</code>.
e.g. With values 16200 and 16300 for base ports, 99 additional RegionServers can be supported, on a server.
The following command starts four additional RegionServers, running on sequential ports starting at 16022/16032 (base ports 16020/16030 plus 2).</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ .bin/local-regionservers.sh start 2 3 4 5</pre>
</div>
</div>
<div class="paragraph">
<p>To stop a RegionServer manually, use the <code>local-regionservers.sh</code> command with the <code>stop</code> parameter and the offset of the server to stop.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ .bin/local-regionservers.sh stop 3</pre>
</div>
</div>
</li>
<li>
<p>Stop HBase.</p>
<div class="paragraph">
<p>You can stop HBase the same way as in the <a href="#quickstart">quickstart</a> procedure, using the <em>bin/stop-hbase.sh</em> command.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="quickstart_fully_distributed"><a class="anchor" href="#quickstart_fully_distributed"></a>2.4. Advanced - Fully Distributed</h3>
<div class="paragraph">
<p>In reality, you need a fully-distributed configuration to fully test HBase and to use it in real-world scenarios.
In a distributed configuration, the cluster contains multiple nodes, each of which runs one or more HBase daemon.
These include primary and backup Master instances, multiple ZooKeeper nodes, and multiple RegionServer nodes.</p>
</div>
<div class="paragraph">
<p>This advanced quickstart adds two more nodes to your cluster.
The architecture will be as follows:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 1. Distributed Cluster Demo Architecture</caption>
<colgroup>
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Node Name</th>
<th class="tableblock halign-left valign-top">Master</th>
<th class="tableblock halign-left valign-top">ZooKeeper</th>
<th class="tableblock halign-left valign-top">RegionServer</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">node-a.example.com</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">no</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">node-b.example.com</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">backup</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">node-c.example.com</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">no</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>This quickstart assumes that each node is a virtual machine and that they are all on the same network.
It builds upon the previous quickstart, <a href="#quickstart_pseudo">Pseudo-Distributed Local Install</a>, assuming that the system you configured in that procedure is now <code>node-a</code>.
Stop HBase on <code>node-a</code> before continuing.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Be sure that all the nodes have full access to communicate, and that no firewall rules are in place which could prevent them from talking to each other.
If you see any errors like <code>no route to host</code>, check your firewall.
</td>
</tr>
</table>
</div>
<div id="passwordless.ssh.quickstart" class="paragraph">
<div class="title">Procedure: Configure Passwordless SSH Access</div>
<p><code>node-a</code> needs to be able to log into <code>node-b</code> and <code>node-c</code> (and to itself) in order to start the daemons.
The easiest way to accomplish this is to use the same username on all hosts, and configure password-less SSH login from <code>node-a</code> to each of the others.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>On <code>node-a</code>, generate a key pair.</p>
<div class="paragraph">
<p>While logged in as the user who will run HBase, generate a SSH key pair, using the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ssh-keygen -t rsa</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the command succeeds, the location of the key pair is printed to standard output.
The default name of the public key is <em>id_rsa.pub</em>.</p>
</div>
</li>
<li>
<p>Create the directory that will hold the shared keys on the other nodes.</p>
<div class="paragraph">
<p>On <code>node-b</code> and <code>node-c</code>, log in as the HBase user and create a <em>.ssh/</em> directory in the user&#8217;s home directory, if it does not already exist.
If it already exists, be aware that it may already contain other keys.</p>
</div>
</li>
<li>
<p>Copy the public key to the other nodes.</p>
<div class="paragraph">
<p>Securely copy the public key from <code>node-a</code> to each of the nodes, by using the <code>scp</code> or some other secure means.
On each of the other nodes, create a new file called <em>.ssh/authorized_keys</em> <em>if it does
not already exist</em>, and append the contents of the <em>id_rsa.pub</em> file to the end of it.
Note that you also need to do this for <code>node-a</code> itself.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ cat id_rsa.pub &gt;&gt; ~/.ssh/authorized_keys</pre>
</div>
</div>
</li>
<li>
<p>Test password-less login.</p>
<div class="paragraph">
<p>If you performed the procedure correctly, you should not be prompted for a password when you SSH from <code>node-a</code> to either of the other nodes using the same username.</p>
</div>
</li>
<li>
<p>Since <code>node-b</code> will run a backup Master, repeat the procedure above, substituting <code>node-b</code> everywhere you see <code>node-a</code>.
Be sure not to overwrite your existing <em>.ssh/authorized_keys</em> files, but concatenate the new key onto the existing file using the <code>&gt;&gt;</code> operator rather than the <code>&gt;</code> operator.</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Procedure: Prepare <code>node-a</code></div>
<p><code>node-a</code> will run your primary master and ZooKeeper processes, but no RegionServers. Stop the RegionServer from starting on <code>node-a</code>.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Edit <em>conf/regionservers</em> and remove the line which contains <code>localhost</code>. Add lines with the hostnames or IP addresses for <code>node-b</code> and <code>node-c</code>.</p>
<div class="paragraph">
<p>Even if you did want to run a RegionServer on <code>node-a</code>, you should refer to it by the hostname the other servers would use to communicate with it.
In this case, that would be <code>node-a.example.com</code>.
This enables you to distribute the configuration to each node of your cluster any hostname conflicts.
Save the file.</p>
</div>
</li>
<li>
<p>Configure HBase to use <code>node-b</code> as a backup master.</p>
<div class="paragraph">
<p>Create a new file in <em>conf/</em> called <em>backup-masters</em>, and add a new line to it with the hostname for <code>node-b</code>.
In this demonstration, the hostname is <code>node-b.example.com</code>.</p>
</div>
</li>
<li>
<p>Configure ZooKeeper</p>
<div class="paragraph">
<p>In reality, you should carefully consider your ZooKeeper configuration.
You can find out more about configuring ZooKeeper in <a href="#zookeeper">zookeeper</a> section.
This configuration will direct HBase to start and manage a ZooKeeper instance on each node of the cluster.</p>
</div>
<div class="paragraph">
<p>On <code>node-a</code>, edit <em>conf/hbase-site.xml</em> and add the following properties.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.quorum<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>node-a.example.com,node-b.example.com,node-c.example.com<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.dataDir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/usr/local/zookeeper<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</li>
<li>
<p>Everywhere in your configuration that you have referred to <code>node-a</code> as <code>localhost</code>, change the reference to point to the hostname that the other nodes will use to refer to <code>node-a</code>.
In these examples, the hostname is <code>node-a.example.com</code>.</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Procedure: Prepare <code>node-b</code> and <code>node-c</code></div>
<p><code>node-b</code> will run a backup master server and a ZooKeeper instance.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Download and unpack HBase.</p>
<div class="paragraph">
<p>Download and unpack HBase to <code>node-b</code>, just as you did for the standalone and pseudo-distributed quickstarts.</p>
</div>
</li>
<li>
<p>Copy the configuration files from <code>node-a</code> to <code>node-b</code>.and <code>node-c</code>.</p>
<div class="paragraph">
<p>Each node of your cluster needs to have the same configuration information.
Copy the contents of the <em>conf/</em> directory to the <em>conf/</em> directory on <code>node-b</code> and <code>node-c</code>.</p>
</div>
</li>
</ol>
</div>
<div class="olist arabic">
<div class="title">Procedure: Start and Test Your Cluster</div>
<ol class="arabic">
<li>
<p>Be sure HBase is not running on any node.</p>
<div class="paragraph">
<p>If you forgot to stop HBase from previous testing, you will have errors.
Check to see whether HBase is running on any of your nodes by using the <code>jps</code> command.
Look for the processes <code>HMaster</code>, <code>HRegionServer</code>, and <code>HQuorumPeer</code>.
If they exist, kill them.</p>
</div>
</li>
<li>
<p>Start the cluster.</p>
<div class="paragraph">
<p>On <code>node-a</code>, issue the <code>start-hbase.sh</code> command.
Your output will be similar to that below.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/start-hbase.sh
node-c.example.com: starting zookeeper, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-zookeeper-node-c.example.com.out
node-a.example.com: starting zookeeper, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-zookeeper-node-a.example.com.out
node-b.example.com: starting zookeeper, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-zookeeper-node-b.example.com.out
starting master, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-master-node-a.example.com.out
node-c.example.com: starting regionserver, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-regionserver-node-c.example.com.out
node-b.example.com: starting regionserver, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-regionserver-node-b.example.com.out
node-b.example.com: starting master, logging to /home/hbuser/hbase-0.98.3-hadoop2/bin/../logs/hbase-hbuser-master-nodeb.example.com.out</pre>
</div>
</div>
<div class="paragraph">
<p>ZooKeeper starts first, followed by the master, then the RegionServers, and finally the backup masters.</p>
</div>
</li>
<li>
<p>Verify that the processes are running.</p>
<div class="paragraph">
<p>On each node of the cluster, run the <code>jps</code> command and verify that the correct processes are running on each server.
You may see additional Java processes running on your servers as well, if they are used for other purposes.</p>
</div>
<div class="listingblock">
<div class="title"><code>node-a</code> <code>jps</code> Output</div>
<div class="content">
<pre>$ jps
20355 Jps
20071 HQuorumPeer
20137 HMaster</pre>
</div>
</div>
<div class="listingblock">
<div class="title"><code>node-b</code> <code>jps</code> Output</div>
<div class="content">
<pre>$ jps
15930 HRegionServer
16194 Jps
15838 HQuorumPeer
16010 HMaster</pre>
</div>
</div>
<div class="listingblock">
<div class="title"><code>node-c</code> <code>jps</code> Output</div>
<div class="content">
<pre>$ jps
13901 Jps
13639 HQuorumPeer
13737 HRegionServer</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">ZooKeeper Process Name</div>
<div class="paragraph">
<p>The <code>HQuorumPeer</code> process is a ZooKeeper instance which is controlled and started by HBase.
If you use ZooKeeper this way, it is limited to one instance per cluster node and is appropriate for testing only.
If ZooKeeper is run outside of HBase, the process is called <code>QuorumPeer</code>.
For more about ZooKeeper configuration, including using an external ZooKeeper instance with HBase, see <a href="#zookeeper">zookeeper</a> section.</p>
</div>
</td>
</tr>
</table>
</div>
</li>
<li>
<p>Browse to the Web UI.</p>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Web UI Port Changes</div>
<div class="paragraph">
<p>In HBase newer than 0.98.x, the HTTP ports used by the HBase Web UI changed from 60010 for the
Master and 60030 for each RegionServer to 16010 for the Master and 16030 for the RegionServer.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>If everything is set up correctly, you should be able to connect to the UI for the Master
<code><a href="http://node-a.example.com:16010/" class="bare">http://node-a.example.com:16010/</a></code> or the secondary master at <code><a href="http://node-b.example.com:16010/" class="bare">http://node-b.example.com:16010/</a></code>
using a web browser.
If you can connect via <code>localhost</code> but not from another host, check your firewall rules.
You can see the web UI for each of the RegionServers at port 16030 of their IP addresses, or by
clicking their links in the web UI for the Master.</p>
</div>
</li>
<li>
<p>Test what happens when nodes or services disappear.</p>
<div class="paragraph">
<p>With a three-node cluster you have configured, things will not be very resilient.
You can still test the behavior of the primary Master or a RegionServer by killing the associated processes and watching the logs.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_where_to_go_next"><a class="anchor" href="#_where_to_go_next"></a>2.5. Where to go next</h3>
<div class="paragraph">
<p>The next chapter, <a href="#configuration">configuration</a>, gives more information about the different HBase run modes, system requirements for running HBase, and critical configuration areas for setting up a distributed HBase cluster.</p>
</div>
</div>
</div>
</div>
<h1 id="configuration" class="sect0"><a class="anchor" href="#configuration"></a>Apache HBase Configuration</h1>
<div class="openblock partintro">
<div class="content">
This chapter expands upon the <a href="#getting_started">Getting Started</a> chapter to further explain configuration of Apache HBase.
Please read this chapter carefully, especially the <a href="#basic.prerequisites">Basic Prerequisites</a>
to ensure that your HBase testing and deployment goes smoothly.
Familiarize yourself with <a href="#hbase_supported_tested_definitions">Support and Testing Expectations</a> as well.
</div>
</div>
<div class="sect1">
<h2 id="_configuration_files"><a class="anchor" href="#_configuration_files"></a>3. Configuration Files</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Apache HBase uses the same configuration system as Apache Hadoop.
All configuration files are located in the <em>conf/</em> directory, which needs to be kept in sync for each node on your cluster.</p>
</div>
<div class="dlist">
<div class="title">HBase Configuration File Descriptions</div>
<dl>
<dt class="hdlist1"><em>backup-masters</em></dt>
<dd>
<p>Not present by default.
A plain-text file which lists hosts on which the Master should start a backup Master process, one host per line.</p>
</dd>
<dt class="hdlist1"><em>hadoop-metrics2-hbase.properties</em></dt>
<dd>
<p>Used to connect HBase Hadoop&#8217;s Metrics2 framework.
See the <a href="https://wiki.apache.org/hadoop/HADOOP-6728-MetricsV2">Hadoop Wiki entry</a> for more information on Metrics2.
Contains only commented-out examples by default.</p>
</dd>
<dt class="hdlist1"><em>hbase-env.cmd</em> and <em>hbase-env.sh</em></dt>
<dd>
<p>Script for Windows and Linux / Unix environments to set up the working environment for HBase, including the location of Java, Java options, and other environment variables.
The file contains many commented-out examples to provide guidance.</p>
</dd>
<dt class="hdlist1"><em>hbase-policy.xml</em></dt>
<dd>
<p>The default policy configuration file used by RPC servers to make authorization decisions on client requests.
Only used if HBase <a href="#security">security</a> is enabled.</p>
</dd>
<dt class="hdlist1"><em>hbase-site.xml</em></dt>
<dd>
<p>The main HBase configuration file.
This file specifies configuration options which override HBase&#8217;s default configuration.
You can view (but do not edit) the default configuration file at <em>docs/hbase-default.xml</em>.
You can also view the entire effective configuration for your cluster (defaults and overrides) in the <span class="label">HBase Configuration</span> tab of the HBase Web UI.</p>
</dd>
<dt class="hdlist1"><em>log4j.properties</em></dt>
<dd>
<p>Configuration file for HBase logging via <code>log4j</code>.</p>
</dd>
<dt class="hdlist1"><em>regionservers</em></dt>
<dd>
<p>A plain-text file containing a list of hosts which should run a RegionServer in your HBase cluster.
By default this file contains the single entry <code>localhost</code>.
It should contain a list of hostnames or IP addresses, one per line, and should only contain <code>localhost</code> if each node in your cluster will run a RegionServer on its <code>localhost</code> interface.</p>
</dd>
</dl>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Checking XML Validity</div>
<div class="paragraph">
<p>When you edit XML, it is a good idea to use an XML-aware editor to be sure that your syntax is correct and your XML is well-formed.
You can also use the <code>xmllint</code> utility to check that your XML is well-formed.
By default, <code>xmllint</code> re-flows and prints the XML to standard output.
To check for well-formedness and only print output if errors exist, use the command <code>xmllint -noout filename.xml</code>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Keep Configuration In Sync Across the Cluster</div>
<div class="paragraph">
<p>When running in distributed mode, after you make an edit to an HBase configuration, make sure you copy the contents of the <em>conf/</em> directory to all nodes of the cluster.
HBase will not do this for you.
Use <code>rsync</code>, <code>scp</code>, or another secure mechanism for copying the configuration files to your nodes.
For most configurations, a restart is needed for servers to pick up changes. Dynamic configuration is an exception to this, to be described later below.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="basic.prerequisites"><a class="anchor" href="#basic.prerequisites"></a>4. Basic Prerequisites</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This section lists required services and some required system configuration.</p>
</div>
<div id="java" class="paragraph">
<div class="title">Java</div>
<p>The following table summarizes the recommendation of the HBase community wrt deploying on various Java versions.
A <span class="icon green"><i class="fa fa-check-circle"></i></span> symbol is meant to indicate a base level of testing and willingness to help diagnose and address issues you might run into.
Similarly, an entry of <span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span> or <span class="icon red"><i class="fa fa-times-circle"></i></span> generally means that should you run into an issue the community is likely to ask you to change the Java environment before proceeding to help.
In some cases, specific guidance on limitations (e.g. whether compiling / unit tests work, specific operational issues, etc) will also be noted.</p>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Long Term Support JDKs are recommended</div>
<div class="paragraph">
<p>HBase recommends downstream users rely on JDK releases that are marked as Long Term Supported (LTS) either from the OpenJDK project or vendors. As of March 2018 that means Java 8 is the only applicable version and that the next likely version to see testing will be Java 11 near Q3 2018.</p>
</div>
</td>
</tr>
</table>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 2. Java support by release line</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 16.6666%;">
<col style="width: 16.6666%;">
<col style="width: 16.6666%;">
<col style="width: 16.6666%;">
<col style="width: 16.667%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-center valign-middle">HBase Version</th>
<th class="tableblock halign-center valign-middle">JDK 7</th>
<th class="tableblock halign-center valign-middle">JDK 8</th>
<th class="tableblock halign-center valign-middle">JDK 9 (Non-LTS)</th>
<th class="tableblock halign-center valign-middle">JDK 10 (Non-LTS)</th>
<th class="tableblock halign-center valign-middle">JDK 11</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-center valign-middle"><p class="tableblock">2.0+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-20264">HBASE-20264</a></div></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-20264">HBASE-20264</a></div></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-21110">HBASE-21110</a></div></td>
</tr>
<tr>
<td class="tableblock halign-center valign-middle"><p class="tableblock">1.2+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-20264">HBASE-20264</a></div></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-20264">HBASE-20264</a></div></td>
<td class="tableblock halign-center valign-middle"><div class="verse"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span>
<a href="https://issues.apache.org/jira/browse/HBASE-21110">HBASE-21110</a></div></td>
</tr>
</tbody>
</table>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
HBase will neither build nor run with Java 6.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
You must set <code>JAVA_HOME</code> on each node of your cluster. <em>hbase-env.sh</em> provides a handy mechanism to do this.
</td>
</tr>
</table>
</div>
<div id="os" class="dlist">
<div class="title">Operating System Utilities</div>
<dl>
<dt class="hdlist1">ssh</dt>
<dd>
<p>HBase uses the Secure Shell (ssh) command and utilities extensively to communicate between cluster nodes. Each server in the cluster must be running <code>ssh</code> so that the Hadoop and HBase daemons can be managed. You must be able to connect to all nodes via SSH, including the local node, from the Master as well as any backup Master, using a shared key rather than a password. You can see the basic methodology for such a set-up in Linux or Unix systems at "<a href="#passwordless.ssh.quickstart">Procedure: Configure Passwordless SSH Access</a>". If your cluster nodes use OS X, see the section, <a href="https://wiki.apache.org/hadoop/Running_Hadoop_On_OS_X_10.5_64-bit_%28Single-Node_Cluster%29">SSH: Setting up Remote Desktop and Enabling Self-Login</a> on the Hadoop wiki.</p>
</dd>
<dt class="hdlist1">DNS</dt>
<dd>
<p>HBase uses the local hostname to self-report its IP address.</p>
</dd>
<dt class="hdlist1">NTP</dt>
<dd>
<p>The clocks on cluster nodes should be synchronized. A small amount of variation is acceptable, but larger amounts of skew can cause erratic and unexpected behavior. Time synchronization is one of the first things to check if you see unexplained problems in your cluster. It is recommended that you run a Network Time Protocol (NTP) service, or another time-synchronization mechanism on your cluster and that all nodes look to the same service for time synchronization. See the <a href="http://www.tldp.org/LDP/sag/html/basic-ntp-config.html">Basic NTP Configuration</a> at <em class="citetitle">The Linux Documentation Project (TLDP)</em> to set up NTP.</p>
</dd>
</dl>
</div>
<div id="ulimit" class="dlist">
<dl>
<dt class="hdlist1">Limits on Number of Files and Processes (ulimit)</dt>
<dd>
<p>Apache HBase is a database. It requires the ability to open a large number of files at once. Many Linux distributions limit the number of files a single user is allowed to open to <code>1024</code> (or <code>256</code> on older versions of OS X). You can check this limit on your servers by running the command <code>ulimit -n</code> when logged in as the user which runs HBase. See <a href="#trouble.rs.runtime.filehandles">the Troubleshooting section</a> for some of the problems you may experience if the limit is too low. You may also notice errors such as the following:</p>
<div class="listingblock">
<div class="content">
<pre>2010-04-06 03:04:37,542 INFO org.apache.hadoop.hdfs.DFSClient: Exception increateBlockOutputStream java.io.EOFException
2010-04-06 03:04:37,542 INFO org.apache.hadoop.hdfs.DFSClient: Abandoning block blk_-6935524980745310745_1391901</pre>
</div>
</div>
<div class="paragraph">
<p>It is recommended to raise the ulimit to at least 10,000, but more likely 10,240, because the value is usually expressed in multiples of 1024. Each ColumnFamily has at least one StoreFile, and possibly more than six StoreFiles if the region is under load. The number of open files required depends upon the number of ColumnFamilies and the number of regions. The following is a rough formula for calculating the potential number of open files on a RegionServer.</p>
</div>
<div class="listingblock">
<div class="title">Calculate the Potential Number of Open Files</div>
<div class="content">
<pre>(StoreFiles per ColumnFamily) x (regions per RegionServer)</pre>
</div>
</div>
<div class="paragraph">
<p>For example, assuming that a schema had 3 ColumnFamilies per region with an average of 3 StoreFiles per ColumnFamily, and there are 100 regions per RegionServer, the JVM will open <code>3 * 3 * 100 = 900</code> file descriptors, not counting open JAR files, configuration files, and others. Opening a file does not take many resources, and the risk of allowing a user to open too many files is minimal.</p>
</div>
<div class="paragraph">
<p>Another related setting is the number of processes a user is allowed to run at once. In Linux and Unix, the number of processes is set using the <code>ulimit -u</code> command. This should not be confused with the <code>nproc</code> command, which controls the number of CPUs available to a given user. Under load, a <code>ulimit -u</code> that is too low can cause OutOfMemoryError exceptions.</p>
</div>
<div class="paragraph">
<p>Configuring the maximum number of file descriptors and processes for the user who is running the HBase process is an operating system configuration, rather than an HBase configuration. It is also important to be sure that the settings are changed for the user that actually runs HBase. To see which user started HBase, and that user&#8217;s ulimit configuration, look at the first line of the HBase log for that instance.</p>
</div>
<div class="exampleblock">
<div class="title">Example 2. <code>ulimit</code> Settings on Ubuntu</div>
<div class="content">
<div class="paragraph">
<p>To configure ulimit settings on Ubuntu, edit <em>/etc/security/limits.conf</em>, which is a space-delimited file with four columns. Refer to the man page for <em>limits.conf</em> for details about the format of this file. In the following example, the first line sets both soft and hard limits for the number of open files (nofile) to 32768 for the operating system user with the username hadoop. The second line sets the number of processes to 32000 for the same user.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hadoop - nofile 32768
hadoop - nproc 32000</pre>
</div>
</div>
<div class="paragraph">
<p>The settings are only applied if the Pluggable Authentication Module (PAM) environment is directed to use them. To configure PAM to use these limits, be sure that the <em>/etc/pam.d/common-session</em> file contains the following line:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>session required pam_limits.so</pre>
</div>
</div>
</div>
</div>
</dd>
<dt class="hdlist1">Linux Shell</dt>
<dd>
<p>All of the shell scripts that come with HBase rely on the <a href="http://www.gnu.org/software/bash">GNU Bash</a> shell.</p>
</dd>
<dt class="hdlist1">Windows</dt>
<dd>
<p>Running production systems on Windows machines is not recommended.</p>
</dd>
</dl>
</div>
<div class="sect2">
<h3 id="hadoop"><a class="anchor" href="#hadoop"></a>4.1. <a href="https://hadoop.apache.org">Hadoop</a></h3>
<div class="paragraph">
<p>The following table summarizes the versions of Hadoop supported with each version of HBase. Older versions not appearing in this table are considered unsupported and likely missing necessary features, while newer versions are untested but may be suitable.</p>
</div>
<div class="paragraph">
<p>Based on the version of HBase, you should select the most appropriate version of Hadoop.
You can use Apache Hadoop, or a vendor&#8217;s distribution of Hadoop.
No distinction is made here.
See <a href="https://wiki.apache.org/hadoop/Distributions%20and%20Commercial%20Support">the Hadoop wiki</a> for information about vendors of Hadoop.</p>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop 2.x is recommended.</div>
<div class="paragraph">
<p>Hadoop 2.x is faster and includes features, such as short-circuit reads (see <a href="#perf.hdfs.configs.localread">Leveraging local data</a>),
which will help improve your HBase random read profile.
Hadoop 2.x also includes important bug fixes that will improve your overall HBase experience. HBase does not support running with
earlier versions of Hadoop. See the table below for requirements specific to different HBase versions.</p>
</div>
<div class="paragraph">
<p>Hadoop 3.x is still in early access releases and has not yet been sufficiently tested by the HBase community for production use cases.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Use the following legend to interpret this table:</p>
</div>
<div class="ulist">
<div class="title">Hadoop version support matrix</div>
<ul>
<li>
<p><span class="icon green"><i class="fa fa-check-circle"></i></span> = Tested to be fully-functional</p>
</li>
<li>
<p><span class="icon red"><i class="fa fa-times-circle"></i></span> = Known to not be fully-functional, or there are <a href="https://hadoop.apache.org/cve_list.html">CVEs</a> so we drop the support in newer minor releases</p>
</li>
<li>
<p><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span> = Not tested, may/may-not function</p>
</li>
</ul>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2858%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top"></th>
<th class="tableblock halign-center valign-middle">HBase-1.2.x, HBase-1.3.x</th>
<th class="tableblock halign-center valign-middle">HBase-1.4.x</th>
<th class="tableblock halign-center valign-middle">HBase-1.5.x</th>
<th class="tableblock halign-center valign-middle">HBase-2.0.x</th>
<th class="tableblock halign-center valign-middle">HBase-2.1.x</th>
<th class="tableblock halign-center valign-middle">HBase-2.2.x</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.4.x</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.5.x</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.6.0</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.6.1+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.7.0</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.7.1+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.8.[0-2]</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.8.[3-4]</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.8.5+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.9.[0-1]</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-2.9.2+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon yellow"><i class="fa fa-exclamation-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-3.0.[0-2]</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-3.0.3+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-3.1.0</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Hadoop-3.1.1+</p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon red"><i class="fa fa-times-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
<td class="tableblock halign-center valign-middle"><p class="tableblock"><span class="icon green"><i class="fa fa-check-circle"></i></span></p></td>
</tr>
</tbody>
</table>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop Pre-2.6.1 and JDK 1.8 Kerberos</div>
<div class="paragraph">
<p>When using pre-2.6.1 Hadoop versions and JDK 1.8 in a Kerberos environment, HBase server can fail
and abort due to Kerberos keytab relogin error. Late version of JDK 1.7 (1.7.0_80) has the problem too.
Refer to <a href="https://issues.apache.org/jira/browse/HADOOP-10786">HADOOP-10786</a> for additional details.
Consider upgrading to Hadoop 2.6.1+ in this case.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop 2.6.x</div>
<div class="paragraph">
<p>Hadoop distributions based on the 2.6.x line <strong>must</strong> have
<a href="https://issues.apache.org/jira/browse/HADOOP-11710">HADOOP-11710</a> applied if you plan to run
HBase on top of an HDFS Encryption Zone. Failure to do so will result in cluster failure and
data loss. This patch is present in Apache Hadoop releases 2.6.1+.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop 2.y.0 Releases</div>
<div class="paragraph">
<p>Starting around the time of Hadoop version 2.7.0, the Hadoop PMC got into the habit of calling out new minor releases on their major version 2 release line as not stable / production ready. As such, HBase expressly advises downstream users to avoid running on top of these releases. Note that additionally the 2.8.1 release was given the same caveat by the Hadoop PMC. For reference, see the release announcements for <a href="https://s.apache.org/hadoop-2.7.0-announcement">Apache Hadoop 2.7.0</a>, <a href="https://s.apache.org/hadoop-2.8.0-announcement">Apache Hadoop 2.8.0</a>, <a href="https://s.apache.org/hadoop-2.8.1-announcement">Apache Hadoop 2.8.1</a>, and <a href="https://s.apache.org/hadoop-2.9.0-announcement">Apache Hadoop 2.9.0</a>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop 3.0.x Releases</div>
<div class="paragraph">
<p>Hadoop distributions that include the Application Timeline Service feature may cause unexpected versions of HBase classes to be present in the application classpath. Users planning on running MapReduce applications with HBase should make sure that <a href="https://issues.apache.org/jira/browse/YARN-7190">YARN-7190</a> is present in their YARN service (currently fixed in 2.9.1+ and 3.1.0+).</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop 3.1.0 Release</div>
<div class="paragraph">
<p>The Hadoop PMC called out the 3.1.0 release as not stable / production ready. As such, HBase expressly advises downstream users to avoid running on top of this release. For reference, see the <a href="https://s.apache.org/hadoop-3.1.0-announcement">release announcement for Hadoop 3.1.0</a>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Replace the Hadoop Bundled With HBase!</div>
<div class="paragraph">
<p>Because HBase depends on Hadoop, it bundles Hadoop jars under its <em>lib</em> directory.
The bundled jars are ONLY for use in standalone mode.
In distributed mode, it is <em>critical</em> that the version of Hadoop that is out on your cluster match what is under HBase.
Replace the hadoop jars found in the HBase lib directory with the equivalent hadoop jars from the version you are running
on your cluster to avoid version mismatch issues.
Make sure you replace the jars under HBase across your whole cluster.
Hadoop version mismatch issues have various manifestations. Check for mismatch if
HBase appears hung.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="dfs.datanode.max.transfer.threads"><a class="anchor" href="#dfs.datanode.max.transfer.threads"></a>4.1.1. <code>dfs.datanode.max.transfer.threads</code> </h4>
<div class="paragraph">
<p>An HDFS DataNode has an upper bound on the number of files that it will serve at any one time.
Before doing any loading, make sure you have configured Hadoop&#8217;s <em>conf/hdfs-site.xml</em>, setting the <code>dfs.datanode.max.transfer.threads</code> value to at least the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.datanode.max.transfer.threads<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>4096<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Be sure to restart your HDFS after making the above configuration.</p>
</div>
<div class="paragraph">
<p>Not having this configuration in place makes for strange-looking failures.
One manifestation is a complaint about missing blocks.
For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>10/12/08 20:10:31 INFO hdfs.DFSClient: Could not obtain block
blk_XXXXXXXXXXXXXXXXXXXXXX_YYYYYYYY from any node: java.io.IOException: No live nodes
contain current block. Will get new block locations from namenode and retry...</pre>
</div>
</div>
<div class="paragraph">
<p>See also <a href="#casestudies.max.transfer.threads">casestudies.max.transfer.threads</a> and note that this property was previously known as <code>dfs.datanode.max.xcievers</code> (e.g. <a href="http://ccgtech.blogspot.com/2010/02/hadoop-hdfs-deceived-by-xciever.html">Hadoop HDFS: Deceived by Xciever</a>).</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="zookeeper.requirements"><a class="anchor" href="#zookeeper.requirements"></a>4.2. ZooKeeper Requirements</h3>
<div class="paragraph">
<p>ZooKeeper 3.4.x is required.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="standalone_dist"><a class="anchor" href="#standalone_dist"></a>5. HBase run modes: Standalone and Distributed</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase has two run modes: <a href="#standalone">standalone</a> and <a href="#distributed">distributed</a>.
Out of the box, HBase runs in standalone mode.
Whatever your mode, you will need to configure HBase by editing files in the HBase <em>conf</em> directory.
At a minimum, you must edit <span class="code">conf/hbase-env.sh</span> to tell HBase which java to use.
In this file you set HBase environment variables such as the heapsize and other options for the <code>JVM</code>, the preferred location for log files, etc.
Set <span class="var">JAVA_HOME</span> to point at the root of your java install.</p>
</div>
<div class="sect2">
<h3 id="standalone"><a class="anchor" href="#standalone"></a>5.1. Standalone HBase</h3>
<div class="paragraph">
<p>This is the default mode.
Standalone mode is what is described in the <a href="#quickstart">quickstart</a> section.
In standalone mode, HBase does not use HDFS&#8201;&#8212;&#8201;it uses the local filesystem instead&#8201;&#8212;&#8201;and it runs all HBase daemons and a local ZooKeeper all up in the same JVM.
ZooKeeper binds to a well known port so clients may talk to HBase.</p>
</div>
<div class="sect3">
<h4 id="standalone.over.hdfs"><a class="anchor" href="#standalone.over.hdfs"></a>5.1.1. Standalone HBase over HDFS</h4>
<div class="paragraph">
<p>A sometimes useful variation on standalone hbase has all daemons running inside the
one JVM but rather than persist to the local filesystem, instead
they persist to an HDFS instance.</p>
</div>
<div class="paragraph">
<p>You might consider this profile when you are intent on
a simple deploy profile, the loading is light, but the
data must persist across node comings and goings. Writing to
HDFS where data is replicated ensures the latter.</p>
</div>
<div class="paragraph">
<p>To configure this standalone variant, edit your <em>hbase-site.xml</em>
setting <em>hbase.rootdir</em> to point at a directory in your
HDFS instance but then set <em>hbase.cluster.distributed</em>
to <em>false</em>. For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rootdir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hdfs://namenode.example.org:8020/hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.cluster.distributed<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>false<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="distributed"><a class="anchor" href="#distributed"></a>5.2. Distributed</h3>
<div class="paragraph">
<p>Distributed mode can be subdivided into distributed but all daemons run on a single node&#8201;&#8212;&#8201;a.k.a. <em>pseudo-distributed</em>&#8201;&#8212;&#8201;and <em>fully-distributed</em> where the daemons are spread across all nodes in the cluster.
The <em>pseudo-distributed</em> vs. <em>fully-distributed</em> nomenclature comes from Hadoop.</p>
</div>
<div class="paragraph">
<p>Pseudo-distributed mode can run against the local filesystem or it can run against an instance of the <em>Hadoop Distributed File System</em> (HDFS). Fully-distributed mode can ONLY run on HDFS.
See the Hadoop <a href="https://hadoop.apache.org/docs/current/">documentation</a> for how to set up HDFS.
A good walk-through for setting up HDFS on Hadoop 2 can be found at <a href="http://www.alexjf.net/blog/distributed-systems/hadoop-yarn-installation-definitive-guide" class="bare">http://www.alexjf.net/blog/distributed-systems/hadoop-yarn-installation-definitive-guide</a>.</p>
</div>
<div class="sect3">
<h4 id="pseudo"><a class="anchor" href="#pseudo"></a>5.2.1. Pseudo-distributed</h4>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Pseudo-Distributed Quickstart</div>
<div class="paragraph">
<p>A quickstart has been added to the <a href="#quickstart">quickstart</a> chapter.
See <a href="#quickstart_pseudo">quickstart-pseudo</a>.
Some of the information that was originally in this section has been moved there.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>A pseudo-distributed mode is simply a fully-distributed mode run on a single host.
Use this HBase configuration for testing and prototyping purposes only.
Do not use this configuration for production or for performance evaluation.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="fully_dist"><a class="anchor" href="#fully_dist"></a>5.3. Fully-distributed</h3>
<div class="paragraph">
<p>By default, HBase runs in standalone mode.
Both standalone mode and pseudo-distributed mode are provided for the purposes of small-scale testing.
For a production environment, distributed mode is advised.
In distributed mode, multiple instances of HBase daemons run on multiple servers in the cluster.</p>
</div>
<div class="paragraph">
<p>Just as in pseudo-distributed mode, a fully distributed configuration requires that you set the <code>hbase.cluster.distributed</code> property to <code>true</code>.
Typically, the <code>hbase.rootdir</code> is configured to point to a highly-available HDFS filesystem.</p>
</div>
<div class="paragraph">
<p>In addition, the cluster is configured so that multiple cluster nodes enlist as RegionServers, ZooKeeper QuorumPeers, and backup HMaster servers.
These configuration basics are all demonstrated in <a href="#quickstart_fully_distributed">quickstart-fully-distributed</a>.</p>
</div>
<div class="paragraph">
<div class="title">Distributed RegionServers</div>
<p>Typically, your cluster will contain multiple RegionServers all running on different servers, as well as primary and backup Master and ZooKeeper daemons.
The <em>conf/regionservers</em> file on the master server contains a list of hosts whose RegionServers are associated with this cluster.
Each host is on a separate line.
All hosts listed in this file will have their RegionServer processes started and stopped when the master server starts or stops.</p>
</div>
<div class="paragraph">
<div class="title">ZooKeeper and HBase</div>
<p>See the <a href="#zookeeper">ZooKeeper</a> section for ZooKeeper setup instructions for HBase.</p>
</div>
<div class="exampleblock">
<div class="title">Example 3. Example Distributed HBase Cluster</div>
<div class="content">
<div class="paragraph">
<p>This is a bare-bones <em>conf/hbase-site.xml</em> for a distributed HBase cluster.
A cluster that is used for real-world work would contain more custom configuration parameters.
Most HBase configuration directives have default values, which are used unless the value is overridden in the <em>hbase-site.xml</em>.
See "<a href="#config.files">Configuration Files</a>" for more information.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rootdir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hdfs://namenode.example.org:8020/hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.cluster.distributed<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.quorum<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>node-a.example.com,node-b.example.com,node-c.example.com<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>This is an example <em>conf/regionservers</em> file, which contains a list of nodes that should run a RegionServer in the cluster.
These nodes need HBase installed and they need to use the same contents of the <em>conf/</em> directory as the Master server</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">node-a.example.com
node-b.example.com
node-c.example.com</code></pre>
</div>
</div>
<div class="paragraph">
<p>This is an example <em>conf/backup-masters</em> file, which contains a list of each node that should run a backup Master instance.
The backup Master instances will sit idle unless the main Master becomes unavailable.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">node-b.example.com
node-c.example.com</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<div class="title">Distributed HBase Quickstart</div>
<p>See <a href="#quickstart_fully_distributed">quickstart-fully-distributed</a> for a walk-through of a simple three-node cluster configuration with multiple ZooKeeper, backup HMaster, and RegionServer instances.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: HDFS Client Configuration</div>
<ol class="arabic">
<li>
<p>Of note, if you have made HDFS client configuration changes on your Hadoop cluster, such as configuration directives for HDFS clients, as opposed to server-side configurations, you must use one of the following methods to enable HBase to see and use these configuration changes:</p>
<div class="olist loweralpha">
<ol class="loweralpha" type="a">
<li>
<p>Add a pointer to your <code>HADOOP_CONF_DIR</code> to the <code>HBASE_CLASSPATH</code> environment variable in <em>hbase-env.sh</em>.</p>
</li>
<li>
<p>Add a copy of <em>hdfs-site.xml</em> (or <em>hadoop-site.xml</em>) or, better, symlinks, under <em>${HBASE_HOME}/conf</em>, or</p>
</li>
<li>
<p>if only a small set of HDFS client configurations, add them to <em>hbase-site.xml</em>.</p>
</li>
</ol>
</div>
</li>
</ol>
</div>
<div class="paragraph">
<p>An example of such an HDFS client configuration is <code>dfs.replication</code>.
If for example, you want to run with a replication factor of 5, HBase will create files with the default of 3 unless you do the above to make the configuration available to HBase.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="confirm"><a class="anchor" href="#confirm"></a>6. Running and Confirming Your Installation</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Make sure HDFS is running first.
Start and stop the Hadoop HDFS daemons by running <em>bin/start-hdfs.sh</em> over in the <code>HADOOP_HOME</code> directory.
You can ensure it started properly by testing the <code>put</code> and <code>get</code> of files into the Hadoop filesystem.
HBase does not normally use the MapReduce or YARN daemons. These do not need to be started.</p>
</div>
<div class="paragraph">
<p><em>If</em> you are managing your own ZooKeeper, start it and confirm it&#8217;s running, else HBase will start up ZooKeeper for you as part of its start process.</p>
</div>
<div class="paragraph">
<p>Start HBase with the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>bin/start-hbase.sh</pre>
</div>
</div>
<div class="paragraph">
<p>Run the above from the <code>HBASE_HOME</code> directory.</p>
</div>
<div class="paragraph">
<p>You should now have a running HBase instance.
HBase logs can be found in the <em>logs</em> subdirectory.
Check them out especially if HBase had trouble starting.</p>
</div>
<div class="paragraph">
<p>HBase also puts up a UI listing vital attributes.
By default it&#8217;s deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational HTTP server at port 16030). If the Master is running on a host named <code>master.example.org</code> on the default port, point your browser at http://master.example.org:16010 to see the web interface.</p>
</div>
<div class="paragraph">
<p>Once HBase has started, see the <a href="#shell_exercises">shell exercises</a> section for how to create tables, add data, scan your insertions, and finally disable and drop your tables.</p>
</div>
<div class="paragraph">
<p>To stop HBase after exiting the HBase shell enter</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/stop-hbase.sh
stopping hbase...............</pre>
</div>
</div>
<div class="paragraph">
<p>Shutdown can take a moment to complete.
It can take longer if your cluster is comprised of many machines.
If you are running a distributed operation, be sure to wait until HBase has shut down completely before stopping the Hadoop daemons.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="config.files"><a class="anchor" href="#config.files"></a>7. Default Configuration</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="hbase.site"><a class="anchor" href="#hbase.site"></a>7.1. <em>hbase-site.xml</em> and <em>hbase-default.xml</em></h3>
<div class="paragraph">
<p>Just as in Hadoop where you add site-specific HDFS configuration to the <em>hdfs-site.xml</em> file, for HBase, site specific customizations go into the file <em>conf/hbase-site.xml</em>.
For the list of configurable properties, see <a href="#hbase_default_configurations">hbase default configurations</a> below or view the raw <em>hbase-default.xml</em> source file in the HBase source code at <em>src/main/resources</em>.</p>
</div>
<div class="paragraph">
<p>Not all configuration options make it out to <em>hbase-default.xml</em>.
Some configurations would only appear in source code; the only way to identify these changes are through code review.</p>
</div>
<div class="paragraph">
<p>Currently, changes here will require a cluster restart for HBase to notice the change.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbase_default_configurations"><a class="anchor" href="#hbase_default_configurations"></a>7.2. HBase Default Configuration</h3>
<div class="paragraph">
<p>The documentation below is generated using the default hbase configuration file, <em>hbase-default.xml</em>, as source.</p>
</div>
<div id="hbase.tmp.dir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.tmp.dir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Temporary directory on the local filesystem. Change this setting to point to a location more permanent than '/tmp', the usual resolve for java.io.tmpdir, as the '/tmp' directory is cleared on machine restart.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${java.io.tmpdir}/hbase-${user.name}</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rootdir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rootdir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The directory shared by region servers and into which HBase persists. The URL should be 'fully-qualified' to include the filesystem scheme. For example, to specify the HDFS directory '/hbase' where the HDFS instance&#8217;s namenode is running at namenode.example.org on port 9000, set this value to: hdfs://namenode.example.org:9000/hbase. By default, we write to whatever ${hbase.tmp.dir} is set too&#8201;&#8212;&#8201;usually /tmp&#8201;&#8212;&#8201;so change this configuration or else all data will be lost on machine restart.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.tmp.dir}/hbase</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.cluster.distributed" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.cluster.distributed</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The mode the cluster will be in. Possible values are false for standalone mode and true for distributed mode. If false, startup will run all HBase and ZooKeeper daemons together in the one JVM.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.quorum" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.quorum</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Comma separated list of servers in the ZooKeeper ensemble (This config. should have been named hbase.zookeeper.ensemble). For example, "host1.mydomain.com,host2.mydomain.com,host3.mydomain.com". By default this is set to localhost for local and pseudo-distributed modes of operation. For a fully-distributed setup, this should be set to a full list of ZooKeeper ensemble servers. If HBASE_MANAGES_ZK is set in hbase-env.sh this is the list of servers which hbase will start/stop ZooKeeper on as part of cluster start/stop. Client-side, we will take this list of ensemble members and put it together with the hbase.zookeeper.property.clientPort config. and pass it into zookeeper constructor as the connectString parameter.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>localhost</code></p>
</div>
</dd>
</dl>
</div>
<div id="zookeeper.recovery.retry.maxsleeptime" class="dlist">
<dl>
<dt class="hdlist1"><code>zookeeper.recovery.retry.maxsleeptime</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Max sleep time before retry zookeeper operations in milliseconds, a max time is needed here so that sleep time won&#8217;t grow unboundedly</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>60000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.local.dir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.local.dir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Directory on the local filesystem to be used as a local storage.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.tmp.dir}/local/</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The port the HBase Master should bind to.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.info.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.info.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The port for the HBase Master web UI. Set to -1 if you do not want a UI instance run.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16010</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.info.bindAddress" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.info.bindAddress</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The bind address for the HBase Master web UI</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.0.0.0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.logcleaner.plugins" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.logcleaner.plugins</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of BaseLogCleanerDelegate invoked by the LogsCleaner service. These WAL cleaners are called in order, so put the cleaner that prunes the most files in front. To implement your own BaseLogCleanerDelegate, just put it in HBase&#8217;s classpath and add the fully qualified class name here. Always add the above default log cleaners in the list.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveProcedureWALCleaner</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.logcleaner.ttl" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.logcleaner.ttl</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>How long a WAL remain in the archive ({hbase.rootdir}/oldWALs) directory, after which it will be cleaned by a Master thread. The value is in milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>600000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.procedurewalcleaner.ttl" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.procedurewalcleaner.ttl</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>How long a Procedure WAL will remain in the archive directory, after which it will be cleaned by a Master thread. The value is in milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>604800000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.hfilecleaner.plugins" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.hfilecleaner.plugins</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of BaseHFileCleanerDelegate invoked by the HFileCleaner service. These HFiles cleaners are called in order, so put the cleaner that prunes the most files in front. To implement your own BaseHFileCleanerDelegate, just put it in HBase&#8217;s classpath and add the fully qualified class name here. Always add the above default log cleaners in the list as they will be overwritten in hbase-site.xml.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.infoserver.redirect" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.infoserver.redirect</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Whether or not the Master listens to the Master web UI port (hbase.master.info.port) and redirects requests to the web UI server shared by the Master and RegionServer. Config. makes sense when Master is serving Regions (not the default).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.fileSplitTimeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.fileSplitTimeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Splitting a region, how long to wait on the file-splitting step before aborting the attempt. Default: 600000. This setting used to be known as hbase.regionserver.fileSplitTimeout in hbase-1.x. Split is now run master-side hence the rename (If a 'hbase.master.fileSplitTimeout' setting found, will use it to prime the current 'hbase.master.fileSplitTimeout' Configuration.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>600000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The port the HBase RegionServer binds to.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16020</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.info.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.info.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The port for the HBase RegionServer web UI Set to -1 if you do not want the RegionServer UI to run.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16030</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.info.bindAddress" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.info.bindAddress</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The address for the HBase RegionServer web UI</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.0.0.0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.info.port.auto" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.info.port.auto</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Whether or not the Master or RegionServer UI should search for a port to bind to. Enables automatic port search if hbase.regionserver.info.port is already in use. Useful for testing, turned off by default.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.handler.count" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.handler.count</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Count of RPC Listener instances spun up on RegionServers. Same property is used by the Master for count of master handlers. Too many handlers can be counter-productive. Make it a multiple of CPU count. If mostly read-only, handlers count close to cpu count does well. Start with twice the CPU count and tune from there.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>30</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.server.callqueue.handler.factor" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.server.callqueue.handler.factor</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Factor to determine the number of call queues. A value of 0 means a single queue shared between all the handlers. A value of 1 means that each handler has its own queue.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.1</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.server.callqueue.read.ratio" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.server.callqueue.read.ratio</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Split the call queues into read and write queues. The specified interval (which should be between 0.0 and 1.0) will be multiplied by the number of call queues. A value of 0 indicate to not split the call queues, meaning that both read and write requests will be pushed to the same set of queues. A value lower than 0.5 means that there will be less read queues than write queues. A value of 0.5 means there will be the same number of read and write queues. A value greater than 0.5 means that there will be more read queues than write queues. A value of 1.0 means that all the queues except one are used to dispatch read requests. Example: Given the total number of call queues being 10 a read.ratio of 0 means that: the 10 queues will contain both read/write requests. a read.ratio of 0.3 means that: 3 queues will contain only read requests and 7 queues will contain only write requests. a read.ratio of 0.5 means that: 5 queues will contain only read requests and 5 queues will contain only write requests. a read.ratio of 0.8 means that: 8 queues will contain only read requests and 2 queues will contain only write requests. a read.ratio of 1 means that: 9 queues will contain only read requests and 1 queues will contain only write requests.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.server.callqueue.scan.ratio" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.server.callqueue.scan.ratio</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Given the number of read call queues, calculated from the total number of call queues multiplied by the callqueue.read.ratio, the scan.ratio property will split the read call queues into small-read and long-read queues. A value lower than 0.5 means that there will be less long-read queues than short-read queues. A value of 0.5 means that there will be the same number of short-read and long-read queues. A value greater than 0.5 means that there will be more long-read queues than short-read queues A value of 0 or 1 indicate to use the same set of queues for gets and scans. Example: Given the total number of read call queues being 8 a scan.ratio of 0 or 1 means that: 8 queues will contain both long and short read requests. a scan.ratio of 0.3 means that: 2 queues will contain only long-read requests and 6 queues will contain only short-read requests. a scan.ratio of 0.5 means that: 4 queues will contain only long-read requests and 4 queues will contain only short-read requests. a scan.ratio of 0.8 means that: 6 queues will contain only long-read requests and 2 queues will contain only short-read requests.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.msginterval" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.msginterval</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Interval between messages from the RegionServer to Master in milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.logroll.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.logroll.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Period at which we will roll the commit log regardless of how many edits it has.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3600000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.logroll.errors.tolerated" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.logroll.errors.tolerated</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The number of consecutive WAL close errors we will allow before triggering a server abort. A setting of 0 will cause the region server to abort if closing the current WAL writer fails during log rolling. Even a small value (2 or 3) will allow a region server to ride over transient HDFS errors.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.hlog.reader.impl" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.hlog.reader.impl</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The WAL file reader implementation.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.hlog.writer.impl" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.hlog.writer.impl</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The WAL file writer implementation.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.global.memstore.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.global.memstore.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum size of all memstores in a region server before new updates are blocked and flushes are forced. Defaults to 40% of heap (0.4). Updates are blocked and flushes are forced until size of all memstores in a region server hits hbase.regionserver.global.memstore.size.lower.limit. The default value in this configuration has been intentionally left empty in order to honor the old hbase.regionserver.global.memstore.upperLimit property if present.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.global.memstore.size.lower.limit" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.global.memstore.size.lower.limit</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum size of all memstores in a region server before flushes are forced. Defaults to 95% of hbase.regionserver.global.memstore.size (0.95). A 100% value for this value causes the minimum possible flushing to occur when updates are blocked due to memstore limiting. The default value in this configuration has been intentionally left empty in order to honor the old hbase.regionserver.global.memstore.lowerLimit property if present.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.systemtables.compacting.memstore.type" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.systemtables.compacting.memstore.type</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Determines the type of memstore to be used for system tables like META, namespace tables etc. By default NONE is the type and hence we use the default memstore for all the system tables. If we need to use compacting memstore for system tables then set this property to BASIC/EAGER</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>NONE</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.optionalcacheflushinterval" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.optionalcacheflushinterval</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum amount of time an edit lives in memory before being automatically flushed. Default 1 hour. Set it to 0 to disable automatic flushing.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3600000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.dns.interface" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.dns.interface</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The name of the Network Interface from which a region server should report its IP address.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>default</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.dns.nameserver" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.dns.nameserver</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The host name or IP address of the name server (DNS) which a region server should use to determine the host name used by the master for communication and display purposes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>default</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.region.split.policy" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.region.split.policy</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A split policy determines when a region should be split. The various other split policies that are available currently are BusyRegionSplitPolicy, ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy, DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and SteppingSplitPolicy. DisabledRegionSplitPolicy blocks manual region splitting.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.regionserver.SteppingSplitPolicy</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.regionSplitLimit" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.regionSplitLimit</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Limit for the number of regions after which no more region splitting should take place. This is not hard limit for the number of regions but acts as a guideline for the regionserver to stop splitting after a certain limit. Default is set to 1000.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1000</code></p>
</div>
</dd>
</dl>
</div>
<div id="zookeeper.session.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>zookeeper.session.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>ZooKeeper session timeout in milliseconds. It is used in two different ways. First, this value is used in the ZK client that HBase uses to connect to the ensemble. It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See <a href="https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions" class="bare">https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions</a>. For example, if an HBase region server connects to a ZK ensemble that&#8217;s also managed by HBase, then the session timeout will be the one specified by this configuration. But, a region server that connects to an ensemble managed with a different configuration will be subjected that ensemble&#8217;s maxSessionTimeout. So, even though HBase might propose using 90 seconds, the ensemble can have a max timeout lower than this and it will take precedence. The current default maxSessionTimeout that ZK ships with is 40 seconds, which is lower than HBase&#8217;s.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>90000</code></p>
</div>
</dd>
</dl>
</div>
<div id="zookeeper.znode.parent" class="dlist">
<dl>
<dt class="hdlist1"><code>zookeeper.znode.parent</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Root ZNode for HBase in ZooKeeper. All of HBase&#8217;s ZooKeeper files that are configured with a relative path will go under this node. By default, all of HBase&#8217;s ZooKeeper file paths are configured with a relative path, so they will all go under this directory unless changed.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>/hbase</code></p>
</div>
</dd>
</dl>
</div>
<div id="zookeeper.znode.acl.parent" class="dlist">
<dl>
<dt class="hdlist1"><code>zookeeper.znode.acl.parent</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Root ZNode for access control lists.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>acl</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.dns.interface" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.dns.interface</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The name of the Network Interface from which a ZooKeeper server should report its IP address.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>default</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.dns.nameserver" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.dns.nameserver</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The host name or IP address of the name server (DNS) which a ZooKeeper server should use to determine the host name used by the master for communication and display purposes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>default</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.peerport" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.peerport</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Port used by ZooKeeper peers to talk to each other. See <a href="https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper" class="bare">https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper</a> for more information.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2888</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.leaderport" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.leaderport</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Port used by ZooKeeper for leader election. See <a href="https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper" class="bare">https://zookeeper.apache.org/doc/r3.4.10/zookeeperStarted.html#sc_RunningReplicatedZooKeeper</a> for more information.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3888</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.property.initLimit" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.property.initLimit</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Property from ZooKeeper&#8217;s config zoo.cfg. The number of ticks that the initial synchronization phase can take.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.property.syncLimit" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.property.syncLimit</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Property from ZooKeeper&#8217;s config zoo.cfg. The number of ticks that can pass between sending a request and getting an acknowledgment.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>5</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.property.dataDir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.property.dataDir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Property from ZooKeeper&#8217;s config zoo.cfg. The directory where the snapshot is stored.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.tmp.dir}/zookeeper</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.property.clientPort" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.property.clientPort</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Property from ZooKeeper&#8217;s config zoo.cfg. The port at which the clients will connect.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2181</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.zookeeper.property.maxClientCnxns" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.zookeeper.property.maxClientCnxns</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Property from ZooKeeper&#8217;s config zoo.cfg. Limit on number of concurrent connections (at the socket level) that a single client, identified by IP address, may make to a single member of the ZooKeeper ensemble. Set high to avoid zk connection issues running standalone and pseudo-distributed.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>300</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.write.buffer" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.write.buffer</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Default size of the BufferedMutator write buffer in bytes. A bigger buffer takes more memory&#8201;&#8212;&#8201;on both the client and server side since server instantiates the passed write buffer to process it&#8201;&#8212;&#8201;but a larger buffer size reduces the number of RPCs made. For an estimate of server-side memory-used, evaluate hbase.client.write.buffer * hbase.regionserver.handler.count</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2097152</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.pause" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.pause</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>General client pause value. Used mostly as value to wait before running a retry of a failed get, region lookup, etc. See hbase.client.retries.number for description of how we backoff from this initial pause amount and how this pause works w/ retries.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>100</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.pause.cqtbe" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.pause.cqtbe</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Whether or not to use a special client pause for CallQueueTooBigException (cqtbe). Set this property to a higher value than hbase.client.pause if you observe frequent CQTBE from the same RegionServer and the call queue there keeps full</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.retries.number" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.retries.number</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum retries. Used as maximum for all retryable operations such as the getting of a cell&#8217;s value, starting a row update, etc. Retry interval is a rough function based on hbase.client.pause. At first we retry at this interval but then with backoff, we pretty quickly reach retrying every ten seconds. See HConstants#RETRY_BACKOFF for how the backup ramps up. Change this setting and hbase.client.pause to suit your workload.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>15</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.max.total.tasks" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.max.total.tasks</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of concurrent mutation tasks a single HTable instance will send to the cluster.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>100</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.max.perserver.tasks" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.max.perserver.tasks</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of concurrent mutation tasks a single HTable instance will send to a single region server.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.max.perregion.tasks" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.max.perregion.tasks</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of concurrent mutation tasks the client will maintain to a single Region. That is, if there is already hbase.client.max.perregion.tasks writes in progress for this region, new puts won&#8217;t be sent to this region until some writes finishes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.perserver.requests.threshold" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.perserver.requests.threshold</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The max number of concurrent pending requests for one server in all client threads (process level). Exceeding requests will be thrown ServerTooBusyException immediately to prevent user&#8217;s threads being occupied and blocked by only one slow region server. If you use a fix number of threads to access HBase in a synchronous way, set this to a suitable value which is related to the number of threads will help you. See <a href="https://issues.apache.org/jira/browse/HBASE-16388" class="bare">https://issues.apache.org/jira/browse/HBASE-16388</a> for details.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2147483647</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.scanner.caching" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.scanner.caching</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Number of rows that we try to fetch when calling next on a scanner if it is not served from (local, client) memory. This configuration works together with hbase.client.scanner.max.result.size to try and use the network efficiently. The default value is Integer.MAX_VALUE by default so that the network will fill the chunk size defined by hbase.client.scanner.max.result.size rather than be limited by a particular number of rows since the size of rows varies table to table. If you know ahead of time that you will not require more than a certain number of rows from a scan, this configuration should be set to that row limit via Scan#setCaching. Higher caching values will enable faster scanners but will eat up more memory and some calls of next may take longer and longer times when the cache is empty. Do not set this value such that the time between invocations is greater than the scanner timeout; i.e. hbase.client.scanner.timeout.period</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2147483647</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.keyvalue.maxsize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.keyvalue.maxsize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Specifies the combined maximum allowed size of a KeyValue instance. This is to set an upper boundary for a single entry saved in a storage file. Since they cannot be split it helps avoiding that a region cannot be split any further because the data is too large. It seems wise to set this to a fraction of the maximum region size. Setting it to zero or less disables the check.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10485760</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.server.keyvalue.maxsize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.server.keyvalue.maxsize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum allowed size of an individual cell, inclusive of value and all key components. A value of 0 or less disables the check. The default value is 10MB. This is a safety setting to protect the server from OOM situations.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10485760</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.scanner.timeout.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.scanner.timeout.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Client scanner lease period in milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>60000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.localityCheck.threadPoolSize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.localityCheck.threadPoolSize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.bulkload.retries.number" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.bulkload.retries.number</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum retries. This is maximum number of iterations to atomic bulk loads are attempted in the face of splitting operations 0 means never give up.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.balancer.maxRitPercent" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.balancer.maxRitPercent</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The max percent of regions in transition when balancing. The default value is 1.0. So there are no balancer throttling. If set this config to 0.01, It means that there are at most 1% regions in transition when balancing. Then the cluster&#8217;s availability is at least 99% when balancing.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1.0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.balancer.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.balancer.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Period at which the region balancer runs in the Master.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>300000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.normalizer.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.normalizer.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Period at which the region normalizer runs in the Master.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>300000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.normalizer.min.region.count" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.normalizer.min.region.count</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>configure the minimum number of regions</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regions.slop" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regions.slop</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Rebalance if any regionserver has average + (average * slop) regions. The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load balancer), while the default is 0.2 in other load balancers (i.e., SimpleLoadBalancer).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.001</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.server.thread.wakefrequency" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.server.thread.wakefrequency</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>In master side, this config is the period used for FS related behaviors: checking if hdfs is out of safe mode, setting or checking hbase.version file, setting or checking hbase.id file. Using default value should be fine. In regionserver side, this config is used in several places: flushing check interval, compaction check interval, wal rolling check interval. Specially, admin can tune flushing and compaction check interval by hbase.regionserver.flush.check.period and hbase.regionserver.compaction.check.period. (in milliseconds)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.flush.check.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.flush.check.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>It determines the flushing check period of PeriodicFlusher in regionserver. If unset, it uses hbase.server.thread.wakefrequency as default value. (in milliseconds)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.server.thread.wakefrequency}</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.compaction.check.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.compaction.check.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>It determines the compaction check period of CompactionChecker in regionserver. If unset, it uses hbase.server.thread.wakefrequency as default value. (in milliseconds)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.server.thread.wakefrequency}</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.server.versionfile.writeattempts" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.server.versionfile.writeattempts</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>How many times to retry attempting to write a version file before just aborting. Each attempt is separated by the hbase.server.thread.wakefrequency milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.memstore.flush.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.memstore.flush.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Memstore will be flushed to disk if size of the memstore exceeds this number of bytes. Value is checked by a thread that runs every hbase.server.thread.wakefrequency.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>134217728</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.percolumnfamilyflush.size.lower.bound.min" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.percolumnfamilyflush.size.lower.bound.min</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If FlushLargeStoresPolicy is used and there are multiple column families, then every time that we hit the total memstore limit, we find out all the column families whose memstores exceed a "lower bound" and only flush them while retaining the others in memory. The "lower bound" will be "hbase.hregion.memstore.flush.size / column_family_number" by default unless value of this property is larger than that. If none of the families have their memstore size more than lower bound, all the memstores will be flushed (just as usual).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16777216</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.preclose.flush.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.preclose.flush.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If the memstores in a region are this size or larger when we go to close, run a "pre-flush" to clear out memstores before we put up the region closed flag and take the region offline. On close, a flush is run under the close flag to empty memory. During this time the region is offline and we are not taking on any writes. If the memstore content is large, this flush could take a long time to complete. The preflush is meant to clean out the bulk of the memstore before putting up the close flag and taking the region offline so the flush that runs under the close flag has little to do.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>5242880</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.memstore.block.multiplier" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.memstore.block.multiplier</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Block updates if memstore has hbase.hregion.memstore.block.multiplier times hbase.hregion.memstore.flush.size bytes. Useful preventing runaway memstore during spikes in update traffic. Without an upper-bound, memstore fills such that when it flushes the resultant flush files take a long time to compact or split, or worse, we OOME.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>4</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.memstore.mslab.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.memstore.mslab.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables the MemStore-Local Allocation Buffer, a feature which works to prevent heap fragmentation under heavy write loads. This can reduce the frequency of stop-the-world GC pauses on large heaps.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.memstore.mslab.chunksize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.memstore.mslab.chunksize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum byte size of a chunk in the MemStoreLAB. Unit: bytes</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2097152</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.offheap.global.memstore.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.offheap.global.memstore.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The amount of off-heap memory all MemStores in a RegionServer may use. A value of 0 means that no off-heap memory will be used and all chunks in MSLAB will be HeapByteBuffer, otherwise the non-zero value means how many megabyte of off-heap memory will be used for chunks in MSLAB and all chunks in MSLAB will be DirectByteBuffer. Unit: megabytes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.memstore.mslab.max.allocation" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.memstore.mslab.max.allocation</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximal size of one allocation in the MemStoreLAB, if the desired byte size exceed this threshold then it will be just allocated from JVM heap rather than MemStoreLAB.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>262144</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.max.filesize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.max.filesize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum HFile size. If the sum of the sizes of a region&#8217;s HFiles has grown to exceed this value, the region is split in two.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10737418240</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.majorcompaction" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.majorcompaction</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Time between major compactions, expressed in milliseconds. Set to 0 to disable time-based automatic major compactions. User-requested and size-based major compactions will still run. This value is multiplied by hbase.hregion.majorcompaction.jitter to cause compaction to start at a somewhat-random time during a given window of time. The default value is 7 days, expressed in milliseconds. If major compactions are causing disruption in your environment, you can configure them to run at off-peak times for your deployment, or disable time-based major compactions by setting this parameter to 0, and run major compactions in a cron job or by another external mechanism.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>604800000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hregion.majorcompaction.jitter" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hregion.majorcompaction.jitter</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occur a given amount of time either side of hbase.hregion.majorcompaction. The smaller the number, the closer the compactions will happen to the hbase.hregion.majorcompaction interval.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.50</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compactionThreshold" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compactionThreshold</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If more than this number of StoreFiles exist in any one Store (one StoreFile is written per flush of MemStore), a compaction is run to rewrite all StoreFiles into a single StoreFile. Larger values delay compaction, but when compaction does occur, it takes longer to complete.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.compaction.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.compaction.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enable/disable compactions on by setting true/false. We can further switch compactions dynamically with the compaction_switch shell command.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.flusher.count" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.flusher.count</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The number of flush threads. With fewer threads, the MemStore flushes will be queued. With more threads, the flushes will be executed in parallel, increasing the load on HDFS, and potentially causing more compactions.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.blockingStoreFiles" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.blockingStoreFiles</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If more than this number of StoreFiles exist in any one Store (one StoreFile is written per flush of MemStore), updates are blocked for this region until a compaction is completed, or until hbase.hstore.blockingWaitTime has been exceeded.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.blockingWaitTime" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.blockingWaitTime</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The time for which a region will block updates after reaching the StoreFile limit defined by hbase.hstore.blockingStoreFiles. After this time has elapsed, the region will stop blocking updates even if a compaction has not been completed.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>90000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.min" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.min</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The minimum number of StoreFiles which must be eligible for compaction before compaction can run. The goal of tuning hbase.hstore.compaction.min is to avoid ending up with too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles in a Store, and this is probably not appropriate. If you set this value too high, all the other values will need to be adjusted accordingly. For most cases, the default value is appropriate. In previous versions of HBase, the parameter hbase.hstore.compaction.min was named hbase.hstore.compactionThreshold.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.max" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.max</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of StoreFiles which will be selected for a single minor compaction, regardless of the number of eligible StoreFiles. Effectively, the value of hbase.hstore.compaction.max controls the length of time it takes a single compaction to complete. Setting it larger means that more StoreFiles are included in a compaction. For most cases, the default value is appropriate.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.min.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.min.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy) smaller than this size will always be eligible for minor compaction. HFiles this size or larger are evaluated by hbase.hstore.compaction.ratio to determine if they are eligible. Because this limit represents the "automatic include" limit for all StoreFiles smaller than this value, this value may need to be reduced in write-heavy environments where many StoreFiles in the 1-2 MB range are being flushed, because every StoreFile will be targeted for compaction and the resulting StoreFiles may still be under the minimum size and require further compaction. If this parameter is lowered, the ratio check is triggered more quickly. This addressed some issues seen in earlier versions of HBase but changing this parameter is no longer necessary in most situations. Default: 128 MB expressed in bytes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>134217728</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.max.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.max.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy) larger than this size will be excluded from compaction. The effect of raising hbase.hstore.compaction.max.size is fewer, larger StoreFiles that do not get compacted often. If you feel that compaction is happening too often without much benefit, you can try raising this value. Default: the value of LONG.MAX_VALUE, expressed in bytes.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>9223372036854775807</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.ratio" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.ratio</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>For minor compaction, this ratio is used to determine whether a given StoreFile which is larger than hbase.hstore.compaction.min.size is eligible for compaction. Its effect is to limit compaction of large StoreFiles. The value of hbase.hstore.compaction.ratio is expressed as a floating-point decimal. A large ratio, such as 10, will produce a single giant StoreFile. Conversely, a low value, such as .25, will produce behavior similar to the BigTable compaction algorithm, producing four StoreFiles. A moderate value of between 1.0 and 1.4 is recommended. When tuning this value, you are balancing write costs with read costs. Raising the value (to something like 1.4) will have more write costs, because you will compact larger StoreFiles. However, during reads, HBase will need to seek through fewer StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of Bloom filters. Otherwise, you can lower this value to something like 1.0 to reduce the background cost of writes, and use Bloom filters to control the number of StoreFiles touched during reads. For most cases, the default value is appropriate.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1.2F</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.ratio.offpeak" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.ratio.offpeak</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Allows you to set a different (by default, more aggressive) ratio for determining whether larger StoreFiles are included in compactions during off-peak hours. Works in the same way as hbase.hstore.compaction.ratio. Only applies if hbase.offpeak.start.hour and hbase.offpeak.end.hour are also enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>5.0F</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.time.to.purge.deletes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.time.to.purge.deletes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The amount of time to delay purging of delete markers with future timestamps. If unset, or set to 0, all delete markers, including those with future timestamps, are purged during the next major compaction. Otherwise, a delete marker is kept until the major compaction which occurs after the marker&#8217;s timestamp plus the value of this setting, in milliseconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.offpeak.start.hour" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.offpeak.start.hour</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The start of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set to -1 to disable off-peak.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>-1</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.offpeak.end.hour" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.offpeak.end.hour</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The end of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set to -1 to disable off-peak.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>-1</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.thread.compaction.throttle" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.thread.compaction.throttle</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>There are two different thread pools for compactions, one for large compactions and the other for small compactions. This helps to keep compaction of lean tables (such as hbase:meta) fast. If a compaction is larger than this threshold, it goes into the large compaction pool. In most cases, the default value is appropriate. Default: 2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size (which defaults to 128MB). The value field assumes that the value of hbase.hregion.memstore.flush.size is unchanged from the default.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2684354560</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.majorcompaction.pagecache.drop" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.majorcompaction.pagecache.drop</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Specifies whether to drop pages read/written into the system page cache by major compactions. Setting it to true helps prevent major compactions from polluting the page cache, which is almost always required, especially for clusters with low/moderate memory to storage ratio.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.minorcompaction.pagecache.drop" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.minorcompaction.pagecache.drop</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Specifies whether to drop pages read/written into the system page cache by minor compactions. Setting it to true helps prevent minor compactions from polluting the page cache, which is most beneficial on clusters with low memory to storage ratio or very write heavy clusters. You may want to set it to false under moderate to low write workload when bulk of the reads are on the most recently written data.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.kv.max" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.kv.max</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of KeyValues to read and then write in a batch when flushing or compacting. Set this lower if you have big KeyValues and problems with Out Of Memory Exceptions Set this higher if you have wide, small rows.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.storescanner.parallel.seek.enable" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.storescanner.parallel.seek.enable</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables StoreFileScanner parallel-seeking in StoreScanner, a feature which can reduce response latency under special conditions.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.storescanner.parallel.seek.threads" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.storescanner.parallel.seek.threads</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The default thread pool size if parallel-seeking feature enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hfile.block.cache.policy" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.block.cache.policy</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The eviction policy for the L1 block cache (LRU or TinyLFU).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>LRU</code></p>
</div>
</dd>
</dl>
</div>
<div id="hfile.block.cache.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.block.cache.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Percentage of maximum heap (-Xmx setting) to allocate to block cache used by a StoreFile. Default of 0.4 means allocate 40%. Set to 0 to disable but it&#8217;s not recommended; you need at least enough cache to hold the storefile indices.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.4</code></p>
</div>
</dd>
</dl>
</div>
<div id="hfile.block.index.cacheonwrite" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.block.index.cacheonwrite</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This allows to put non-root multi-level index blocks into the block cache at the time the index is being written.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hfile.index.block.max.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.index.block.max.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>When the size of a leaf-level, intermediate-level, or root-level index block in a multi-level block index grows to this size, the block is written out and a new block is started.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>131072</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.bucketcache.ioengine" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.bucketcache.ioengine</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Where to store the contents of the bucketcache. One of: offheap, file, files, mmap or pmem. If a file or files, set it to file(s):PATH_TO_FILE. mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE. 'pmem' is bucket cache over a file on the persistent memory device. Use pmem:PATH_TO_FILE. See <a href="http://hbase.apache.org/book.html#offheap.blockcache" class="bare">http://hbase.apache.org/book.html#offheap.blockcache</a> for more information.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.throughput.lower.bound" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.throughput.lower.bound</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The target lower bound on aggregate compaction throughput, in bytes/sec. Allows you to tune the minimum available compaction throughput when the PressureAwareCompactionThroughputController throughput controller is active. (It is active by default.)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>52428800</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.compaction.throughput.higher.bound" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.throughput.higher.bound</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The target upper bound on aggregate compaction throughput, in bytes/sec. Allows you to control aggregate compaction throughput demand when the PressureAwareCompactionThroughputController throughput controller is active. (It is active by default.) The maximum throughput will be tuned between the lower and upper bounds when compaction pressure is within the range [0.0, 1.0]. If compaction pressure is 1.0 or greater the higher bound will be ignored until pressure returns to the normal range.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>104857600</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.bucketcache.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.bucketcache.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A float that EITHER represents a percentage of total heap memory size to give to the cache (if &lt; 1.0) OR, it is the total capacity in megabytes of BucketCache. Default: 0.0</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.bucketcache.bucket.sizes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.bucketcache.bucket.sizes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of sizes for buckets for the bucketcache. Can be multiple sizes. List block sizes in order from smallest to largest. The sizes you use will depend on your data access patterns. Must be a multiple of 256 else you will run into 'java.io.IOException: Invalid HFile block magic' when you go to read from cache. If you specify no values here, then you pick up the default bucketsizes set in code (See BucketAllocator#DEFAULT_BUCKET_SIZES).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hfile.format.version" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.format.version</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The HFile format version to use for new files. Version 3 adds support for tags in hfiles (See <a href="http://hbase.apache.org/book.html#hbase.tags" class="bare">http://hbase.apache.org/book.html#hbase.tags</a>). Also see the configuration 'hbase.replication.rpc.codec'.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hfile.block.bloom.cacheonwrite" class="dlist">
<dl>
<dt class="hdlist1"><code>hfile.block.bloom.cacheonwrite</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables cache-on-write for inline blocks of a compound Bloom filter.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="io.storefile.bloom.block.size" class="dlist">
<dl>
<dt class="hdlist1"><code>io.storefile.bloom.block.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The size in bytes of a single block ("chunk") of a compound Bloom filter. This size is approximate, because Bloom blocks can only be inserted at data block boundaries, and the number of keys per data block varies.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>131072</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rs.cacheblocksonwrite" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rs.cacheblocksonwrite</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Whether an HFile block should be added to the block cache when the block is finished.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rpc.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rpc.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This is for the RPC layer to define how long (millisecond) HBase client applications take for a remote call to time out. It uses pings to check connections but will eventually throw a TimeoutException.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>60000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.operation.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.operation.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Operation timeout is a top-level restriction (millisecond) that makes sure a blocking operation in Table will not be blocked more than this. In each operation, if rpc request fails because of timeout or other reason, it will retry until success or throw RetriesExhaustedException. But if the total time being blocking reach the operation timeout before retries exhausted, it will break early and throw SocketTimeoutException.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1200000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.cells.scanned.per.heartbeat.check" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.cells.scanned.per.heartbeat.check</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The number of cells scanned in between heartbeat checks. Heartbeat checks occur during the processing of scans to determine whether or not the server should stop scanning in order to send back a heartbeat message to the client. Heartbeat messages are used to keep the client-server connection alive during long running scans. Small values mean that the heartbeat checks will occur more often and thus will provide a tighter bound on the execution time of the scan. Larger values mean that the heartbeat checks occur less frequently</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rpc.shortoperation.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rpc.shortoperation.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This is another version of "hbase.rpc.timeout". For those RPC operation within cluster, we rely on this configuration to set a short timeout limitation for short operation. For example, short rpc timeout for region server&#8217;s trying to report to active master can benefit quicker master failover process.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.client.tcpnodelay" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.client.tcpnodelay</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set no delay on rpc socket connections. See <a href="http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#getTcpNoDelay(" class="bare">http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#getTcpNoDelay(</a>)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.hostname" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.hostname</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This config is for experts: don&#8217;t set its value unless you really know what you are doing. When set to a non-empty value, this represents the (external facing) hostname for the underlying server. See <a href="https://issues.apache.org/jira/browse/HBASE-12954" class="bare">https://issues.apache.org/jira/browse/HBASE-12954</a> for details.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.hostname.disable.master.reversedns" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.hostname.disable.master.reversedns</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This config is for experts: don&#8217;t set its value unless you really know what you are doing. When set to true, regionserver will use the current node hostname for the servername and HMaster will skip reverse DNS lookup and use the hostname sent by regionserver instead. Note that this config and hbase.regionserver.hostname are mutually exclusive. See <a href="https://issues.apache.org/jira/browse/HBASE-18226" class="bare">https://issues.apache.org/jira/browse/HBASE-18226</a> for more details.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.keytab.file" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.keytab.file</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Full path to the kerberos keytab file to use for logging in the configured HMaster server principal.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.kerberos.principal" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.kerberos.principal</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Ex. "hbase/_HOST@EXAMPLE.COM". The kerberos principal name that should be used to run the HMaster process. The principal name should be in the form: user/hostname@DOMAIN. If "_HOST" is used as the hostname portion, it will be replaced with the actual hostname of the running instance.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.keytab.file" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.keytab.file</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Full path to the kerberos keytab file to use for logging in the configured HRegionServer server principal.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.kerberos.principal" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.kerberos.principal</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Ex. "hbase/_HOST@EXAMPLE.COM". The kerberos principal name that should be used to run the HRegionServer process. The principal name should be in the form: user/hostname@DOMAIN. If "_HOST" is used as the hostname portion, it will be replaced with the actual hostname of the running instance. An entry for this principal must exist in the file specified in hbase.regionserver.keytab.file</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hadoop.policy.file" class="dlist">
<dl>
<dt class="hdlist1"><code>hadoop.policy.file</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The policy configuration file used by RPC servers to make authorization decisions on client requests. Only used when HBase security is enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>hbase-policy.xml</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.superuser" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.superuser</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>List of users or groups (comma-separated), who are allowed full privileges, regardless of stored ACLs, across the cluster. Only used when HBase security is enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.auth.key.update.interval" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.auth.key.update.interval</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The update interval for master key for authentication tokens in servers in milliseconds. Only used when HBase security is enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>86400000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.auth.token.max.lifetime" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.auth.token.max.lifetime</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum lifetime in milliseconds after which an authentication token expires. Only used when HBase security is enabled.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>604800000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.client.fallback-to-simple-auth-allowed" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.client.fallback-to-simple-auth-allowed</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>When a client is configured to attempt a secure connection, but attempts to connect to an insecure server, that server may instruct the client to switch to SASL SIMPLE (unsecure) authentication. This setting controls whether or not the client will accept this instruction from the server. When false (the default), the client will not allow the fallback to SIMPLE authentication, and will abort the connection.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.ipc.server.fallback-to-simple-auth-allowed" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.ipc.server.fallback-to-simple-auth-allowed</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>When a server is configured to require secure connections, it will reject connection attempts from clients using SASL SIMPLE (unsecure) authentication. This setting allows secure servers to accept SASL SIMPLE connections from clients when the client requests. When false (the default), the server will not allow the fallback to SIMPLE authentication, and will reject the connection. WARNING: This setting should ONLY be used as a temporary measure while converting clients over to secure authentication. It MUST BE DISABLED for secure operation.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.display.keys" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.display.keys</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>When this is set to true the webUI and such will display all start/end keys as part of the table details, region names, etc. When this is set to false, the keys are hidden.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coprocessor.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coprocessor.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables or disables coprocessor loading. If 'false' (disabled), any other coprocessor related configuration will be ignored.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coprocessor.user.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coprocessor.user.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables or disables user (aka. table) coprocessor loading. If 'false' (disabled), any table coprocessor attributes in table descriptors will be ignored. If "hbase.coprocessor.enabled" is 'false' this setting has no effect.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coprocessor.region.classes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coprocessor.region.classes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of region observer or endpoint coprocessors that are loaded by default on all tables. For any override coprocessor method, these classes will be called in order. After implementing your own Coprocessor, add it to HBase&#8217;s classpath and add the fully qualified class name here. A coprocessor can also be loaded on demand by setting HTableDescriptor or the HBase shell.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coprocessor.master.classes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coprocessor.master.classes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are loaded by default on the active HMaster process. For any implemented coprocessor methods, the listed classes will be called in order. After implementing your own MasterObserver, just put it in HBase&#8217;s classpath and add the fully qualified class name here.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coprocessor.abortonerror" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coprocessor.abortonerror</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to cause the hosting server (master or regionserver) to abort if a coprocessor fails to load, fails to initialize, or throws an unexpected Throwable object. Setting this to false will allow the server to continue execution but the system wide state of the coprocessor in question will become inconsistent as it will be properly executing in only a subset of servers, so this is most useful for debugging only.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The port for the HBase REST server.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>8080</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.readonly" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.readonly</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Defines the mode the REST server will be started in. Possible values are: false: All HTTP methods are permitted - GET/PUT/POST/DELETE. true: Only the GET method is permitted.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.threads.max" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.threads.max</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of threads of the REST server thread pool. Threads in the pool are reused to process REST requests. This controls the maximum number of requests processed concurrently. It may help to control the memory used by the REST server to avoid OOM issues. If the thread pool is full, incoming requests will be queued up and wait for some free threads.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>100</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.threads.min" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.threads.min</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The minimum number of threads of the REST server thread pool. The thread pool always has at least these number of threads so the REST server is ready to serve incoming requests.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.support.proxyuser" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.support.proxyuser</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enables running the REST server to support proxy-user mode.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.defaults.for.version.skip" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.defaults.for.version.skip</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to skip the 'hbase.defaults.for.version' check. Setting this to true can be useful in contexts other than the other side of a maven generation; i.e. running in an IDE. You&#8217;ll want to set this boolean to true to avoid seeing the RuntimeException complaint: "hbase-default.xml file seems to be for and old version of HBase (\${hbase.version}), this version is X.X.X-SNAPSHOT"</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.table.lock.enable" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.table.lock.enable</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to enable locking the table in zookeeper for schema change operations. Table locking from master prevents concurrent schema modifications to corrupt table state.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.table.max.rowsize" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.table.max.rowsize</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum size of single row in bytes (default is 1 Gb) for Get&#8217;ting or Scan&#8217;ning without in-row scan flag set. If row size exceeds this limit RowTooBigException is thrown to client.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1073741824</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.thrift.minWorkerThreads" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.thrift.minWorkerThreads</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The "core size" of the thread pool. New threads are created on every connection until this many threads are created.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.thrift.maxWorkerThreads" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.thrift.maxWorkerThreads</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum size of the thread pool. When the pending request queue overflows, new threads are created until their number reaches this number. After that, the server starts dropping connections.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.thrift.maxQueuedRequests" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.thrift.maxQueuedRequests</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of pending Thrift connections waiting in the queue. If there are no idle threads in the pool, the server queues requests. Only when the queue overflows, new threads are added, up to hbase.thrift.maxQueuedRequests threads.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.thrift.framed" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.thrift.framed</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Use Thrift TFramedTransport on the server side. This is the recommended transport for thrift servers and requires a similar setting on the client side. Changing this to false will select the default transport, vulnerable to DoS when malformed requests are issued due to THRIFT-601.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.thrift.framed.max_frame_size_in_mb" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.thrift.framed.max_frame_size_in_mb</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Default frame size when using framed transport, in MB</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.thrift.compact" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.thrift.compact</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Use Thrift TCompactProtocol binary serialization protocol.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rootdir.perms" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rootdir.perms</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>FS Permissions for the root data subdirectory in a secure (kerberos) setup. When master starts, it creates the rootdir with this permissions or sets the permissions if it does not match.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>700</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.wal.dir.perms" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.wal.dir.perms</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>FS Permissions for the root WAL directory in a secure(kerberos) setup. When master starts, it creates the WAL dir with this permissions or sets the permissions if it does not match.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>700</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.data.umask.enable" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.data.umask.enable</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Enable, if true, that file permissions should be assigned to the files written by the regionserver</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.data.umask" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.data.umask</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>File permissions that should be used to write data files when hbase.data.umask.enable is true</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to allow snapshots to be taken / restored / cloned.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.restore.take.failsafe.snapshot" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.restore.take.failsafe.snapshot</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to take a snapshot before the restore operation. The snapshot taken will be used in case of failure, to restore the previous state. At the end of the restore operation this snapshot will be deleted</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.restore.failsafe.name" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.restore.failsafe.name</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Name of the failsafe snapshot taken by the restore operation. You can use the {snapshot.name}, {table.name} and {restore.timestamp} variables to create a name based on what you are restoring.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>hbase-failsafe-{snapshot.name}-{restore.timestamp}</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.working.dir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.working.dir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Location where the snapshotting process will occur. The location of the completed snapshots will not change, but the temporary directory where the snapshot process occurs will be set to this location. This can be a separate filesystem than the root directory, for performance increase purposes. See HBASE-21098 for more information</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.server.compactchecker.interval.multiplier" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.server.compactchecker.interval.multiplier</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The number that determines how often we scan to see if compaction is necessary. Normally, compactions are done after some events (such as memstore flush), but if region didn&#8217;t receive a lot of writes for some time, or due to different compaction policies, it may be necessary to check it periodically. The interval between checks is hbase.server.compactchecker.interval.multiplier multiplied by hbase.server.thread.wakefrequency.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.lease.recovery.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.lease.recovery.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>How long we wait on dfs lease recovery in total before giving up.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>900000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.lease.recovery.dfs.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.lease.recovery.dfs.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>How long between dfs recover lease invocations. Should be larger than the sum of the time it takes for the namenode to issue a block recovery command as part of datanode; dfs.heartbeat.interval and the time it takes for the primary datanode, performing block recovery to timeout on a dead datanode; usually dfs.client.socket-timeout. See the end of HBASE-8389 for more.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>64000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.column.max.version" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.column.max.version</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>New column family descriptors will use this value as the default number of versions to keep.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1</code></p>
</div>
</dd>
</dl>
</div>
<div id="dfs.client.read.shortcircuit" class="dlist">
<dl>
<dt class="hdlist1"><code>dfs.client.read.shortcircuit</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If set to true, this configuration parameter enables short-circuit local reads.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="dfs.domain.socket.path" class="dlist">
<dl>
<dt class="hdlist1"><code>dfs.domain.socket.path</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This is a path to a UNIX domain socket that will be used for communication between the DataNode and local HDFS clients, if dfs.client.read.shortcircuit is set to true. If the string "_PORT" is present in this path, it will be replaced by the TCP port of the DataNode. Be careful about permissions for the directory that hosts the shared domain socket; dfsclient will complain if open to other users than the HBase user.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>none</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.dfs.client.read.shortcircuit.buffer.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.dfs.client.read.shortcircuit.buffer.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If the DFSClient configuration dfs.client.read.shortcircuit.buffer.size is unset, we will use what is configured here as the short circuit read default direct byte buffer size. DFSClient native default is 1MB; HBase keeps its HDFS files open so number of file blocks * 1MB soon starts to add up and threaten OOME because of a shortage of direct memory. So, we set it down from the default. Make it &gt; the default hbase block size set in the HColumnDescriptor which is usually 64k.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>131072</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.checksum.verify" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.checksum.verify</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If set to true (the default), HBase verifies the checksums for hfile blocks. HBase writes checksums inline with the data when it writes out hfiles. HDFS (as of this writing) writes checksums to a separate file than the data file necessitating extra seeks. Setting this flag saves some on i/o. Checksum verification by HDFS will be internally disabled on hfile streams when this flag is set. If the hbase-checksum verification fails, we will switch back to using HDFS checksums (so do not disable HDFS checksums! And besides this feature applies to hfiles only, not to WALs). If this parameter is set to false, then hbase will not verify any checksums, instead it will depend on checksum verification being done in the HDFS client.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>true</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.bytes.per.checksum" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.bytes.per.checksum</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Number of bytes in a newly created checksum chunk for HBase-level checksums in hfile blocks.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16384</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.hstore.checksum.algorithm" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.checksum.algorithm</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Name of an algorithm that is used to compute checksums. Possible values are NULL, CRC32, CRC32C.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>CRC32C</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.client.scanner.max.result.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.client.scanner.max.result.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum number of bytes returned when calling a scanner&#8217;s next method. Note that when a single row is larger than this limit the row is still returned completely. The default value is 2MB, which is good for 1ge networks. With faster and/or high latency networks this value should be increased.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>2097152</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.server.scanner.max.result.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.server.scanner.max.result.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Maximum number of bytes returned when calling a scanner&#8217;s next method. Note that when a single row is larger than this limit the row is still returned completely. The default value is 100MB. This is a safety setting to protect the server from OOM situations.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>104857600</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.status.published" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.status.published</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This setting activates the publication by the master of the status of the region server. When a region server dies and its recovery starts, the master will push this information to the client application, to let them cut the connection immediately instead of waiting for a timeout.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.status.publisher.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.status.publisher.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Implementation of the status publication with a multicast message.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.master.ClusterStatusPublisher$MulticastPublisher</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.status.listener.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.status.listener.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Implementation of the status listener with a multicast message.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.status.multicast.address.ip" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.status.multicast.address.ip</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Multicast address to use for the status publication by multicast.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>226.1.1.3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.status.multicast.address.port" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.status.multicast.address.port</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Multicast port to use for the status publication by multicast.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16100</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.dynamic.jars.dir" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.dynamic.jars.dir</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The directory from which the custom filter JARs can be loaded dynamically by the region server without the need to restart. However, an already loaded filter/co-processor class would not be un-loaded. See HBASE-1936 for more details. Does not apply to coprocessors.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>${hbase.rootdir}/lib</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.security.authentication" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.security.authentication</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Controls whether or not secure authentication is enabled for HBase. Possible values are 'simple' (no authentication), and 'kerberos'.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>simple</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.filter.classes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.filter.classes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Servlet filters for REST service.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.rest.filter.GzipFilter</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.loadbalancer.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.loadbalancer.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Class used to execute the regions balancing when the period occurs. See the class comment for more on how it works <a href="http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html" class="bare">http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html</a> It replaces the DefaultLoadBalancer as the default (since renamed as the SimpleLoadBalancer).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.loadbalance.bytable" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.loadbalance.bytable</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Factor Table name when the balancer runs. Default: false.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.normalizer.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.normalizer.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Class used to execute the region normalization when the period occurs. See the class comment for more on how it works <a href="http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html" class="bare">http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html</a></p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest.csrf.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest.csrf.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Set to true to enable protection against cross-site request forgery (CSRF)</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rest-csrf.browser-useragents-regex" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rest-csrf.browser-useragents-regex</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of regular expressions used to match against an HTTP request&#8217;s User-Agent header when protection against cross-site request forgery (CSRF) is enabled for REST server by setting hbase.rest.csrf.enabled to true. If the incoming User-Agent matches any of these regular expressions, then the request is considered to be sent by a browser, and therefore CSRF prevention is enforced. If the request&#8217;s User-Agent does not match any of these regular expressions, then the request is considered to be sent by something other than a browser, such as scripted automation. In this case, CSRF is not a potential attack vector, so the prevention is not enforced. This helps achieve backwards-compatibility with existing automation that has not been updated to send the CSRF prevention header.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code><sup>Mozilla.<strong>,</sup>Opera.</strong></code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.security.exec.permission.checks" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.security.exec.permission.checks</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If this setting is enabled and ACL based access control is active (the AccessController coprocessor is installed either as a system coprocessor or on a table as a table coprocessor) then you must grant all relevant users EXEC privilege if they require the ability to execute coprocessor endpoint calls. EXEC privilege, like any other permission, can be granted globally to a user, or to a user on a per table or per namespace basis. For more information on coprocessor endpoints, see the coprocessor section of the HBase online manual. For more information on granting or revoking permissions using the AccessController, see the security section of the HBase online manual.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.regionserver.classes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.procedure.regionserver.classes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of org.apache.hadoop.hbase.procedure.RegionServerProcedureManager procedure managers that are loaded by default on the active HRegionServer process. The lifecycle methods (init/start/stop) will be called by the active HRegionServer process to perform the specific globally barriered procedure. After implementing your own RegionServerProcedureManager, just put it in HBase&#8217;s classpath and add the fully qualified class name here.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.master.classes" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.procedure.master.classes</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma-separated list of org.apache.hadoop.hbase.procedure.MasterProcedureManager procedure managers that are loaded by default on the active HMaster process. A procedure is identified by its signature and users can use the signature and an instant name to trigger an execution of a globally barriered procedure. After implementing your own MasterProcedureManager, just put it in HBase&#8217;s classpath and add the fully qualified class name here.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p>none</p>
</div>
</dd>
</dl>
</div>
<div id="hbase.coordinated.state.manager.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.coordinated.state.manager.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Fully qualified name of class implementing coordinated state manager.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.storefile.refresh.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.storefile.refresh.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>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.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.region.replica.replication.enabled" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.region.replica.replication.enabled</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>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 &gt; 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.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.http.filter.initializers" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.http.filter.initializers</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>A comma separated list of class names. Each class in the list must extend org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will be initialized. Then, the Filter will be applied to all user facing jsp and servlet web pages. The ordering of the list defines the ordering of the filters. The default StaticUserWebFilter add a user principal as defined by the hbase.http.staticuser.user property.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.http.lib.StaticUserWebFilter</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.security.visibility.mutations.checkauths" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.security.visibility.mutations.checkauths</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>This property if enabled, will check whether the labels in the visibility expression are associated with the user issuing the mutation</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>false</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.http.max.threads" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.http.max.threads</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of threads that the HTTP Server will create in its ThreadPool.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>16</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.replication.rpc.codec" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.replication.rpc.codec</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The codec that is to be used when replication is enabled so that the tags are also replicated. This is used along with HFileV3 which supports tags in them. If tags are not used or if the hfile version used is HFileV2 then KeyValueCodec can be used as the replication codec. Note that using KeyValueCodecWithTags for replication when there are no tags causes no harm.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.codec.KeyValueCodecWithTags</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.replication.source.maxthreads" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.replication.source.maxthreads</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The maximum number of threads any replication source will use for shipping edits to the sinks in parallel. This also limits the number of chunks each replication batch is broken into. Larger values can improve the replication throughput between the master and slave clusters. The default of 10 will rarely need to be changed.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>10</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.http.staticuser.user" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.http.staticuser.user</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The user name to filter as, on static web filters while rendering content. An example use is the HDFS web UI (user to be used for browsing files).</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>dr.stack</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.regionserver.handler.abort.on.error.percent" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.regionserver.handler.abort.on.error.percent</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The percent of region server RPC threads failed to abort RS. -1 Disable aborting; 0 Abort if even a single handler has died; 0.x Abort only when this percent of handlers have died; 1 Abort only all of the handers have died.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.5</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.file.cache.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.file.cache.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Number of opened file handlers to cache. A larger value will benefit reads by providing more file handlers per mob file cache and would reduce frequent file opening and closing. However, if this is set too high, this could lead to a "too many opened file handlers" The default value is 1000.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.cache.evict.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.cache.evict.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The amount of time in seconds before the mob cache evicts cached mob files. The default value is 3600 seconds.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3600</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.cache.evict.remain.ratio" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.cache.evict.remain.ratio</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The ratio (between 0.0 and 1.0) of files that remains cached after an eviction is triggered when the number of cached mob files exceeds the hbase.mob.file.cache.size. The default value is 0.5f.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0.5f</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.mob.ttl.cleaner.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.mob.ttl.cleaner.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The period that ExpiredMobFileCleanerChore runs. The unit is second. The default value is one day. The MOB file name uses only the date part of the file creation time in it. We use this time for deciding TTL expiry of the files. So the removal of TTL expired files might be delayed. The max delay might be 24 hrs.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>86400</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.compaction.mergeable.threshold" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.compaction.mergeable.threshold</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If the size of a mob file is less than this value, it&#8217;s regarded as a small file and needs to be merged in mob compaction. The default value is 1280MB.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1342177280</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.delfile.max.count" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.delfile.max.count</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The max number of del files that is allowed in the mob compaction. In the mob compaction, when the number of existing del files is larger than this value, they are merged until number of del files is not larger this value. The default value is 3.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.compaction.batch.size" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.compaction.batch.size</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The max number of the mob files that is allowed in a batch of the mob compaction. The mob compaction merges the small mob files to bigger ones. If the number of the small files is very large, it could lead to a "too many opened file handlers" in the merge. And the merge has to be split into batches. This value limits the number of mob files that are selected in a batch of the mob compaction. The default value is 100.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>100</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.compaction.chore.period" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.compaction.chore.period</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The period that MobCompactionChore runs. The unit is second. The default value is one week.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>604800</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.compactor.class" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.compactor.class</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Implementation of mob compactor, the default one is PartitionedMobCompactor.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.mob.compaction.threads.max" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.mob.compaction.threads.max</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>The max number of threads used in MobCompactor.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.master.timeout.millis" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.master.timeout.millis</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Timeout for master for the snapshot procedure execution.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>300000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.snapshot.region.timeout" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.snapshot.region.timeout</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Timeout for regionservers to keep threads in snapshot request pool waiting.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>300000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.rpc.rows.warning.threshold" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.rpc.rows.warning.threshold</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Number of rows in a batch operation above which a warning will be logged.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>5000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.wait.on.service.seconds" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.wait.on.service.seconds</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Default is 5 minutes. Make it 30 seconds for tests. See HBASE-19794 for some context.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>30</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.cleaner.snapshot.interval" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.cleaner.snapshot.interval</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Snapshot Cleanup chore interval in milliseconds. The cleanup thread keeps running at this interval to find all snapshots that are expired based on TTL and delete them.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1800000</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.master.snapshot.ttl" class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.master.snapshot.ttl</code></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Default Snapshot TTL to be considered when the user does not specify TTL while creating snapshot. Default value 0 indicates FOREVERE - snapshot should not be automatically deleted until it is manually deleted</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>0</code></p>
</div>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="hbase.env.sh"><a class="anchor" href="#hbase.env.sh"></a>7.3. <em>hbase-env.sh</em></h3>
<div class="paragraph">
<p>Set HBase environment variables in this file.
Examples include options to pass the JVM on start of an HBase daemon such as heap size and garbage collector configs.
You can also set configurations for HBase configuration, log directories, niceness, ssh options, where to locate process pid files, etc.
Open the file at <em>conf/hbase-env.sh</em> and peruse its content.
Each option is fairly well documented.
Add your own environment variables here if you want them read by HBase daemons on startup.</p>
</div>
<div class="paragraph">
<p>Changes here will require a cluster restart for HBase to notice the change.</p>
</div>
</div>
<div class="sect2">
<h3 id="log4j"><a class="anchor" href="#log4j"></a>7.4. <em>log4j.properties</em></h3>
<div class="paragraph">
<p>Edit this file to change rate at which HBase files are rolled and to change the level at which HBase logs messages.</p>
</div>
<div class="paragraph">
<p>Changes here will require a cluster restart for HBase to notice the change though log levels can be changed for particular daemons via the HBase UI.</p>
</div>
</div>
<div class="sect2">
<h3 id="client_dependencies"><a class="anchor" href="#client_dependencies"></a>7.5. Client configuration and dependencies connecting to an HBase cluster</h3>
<div class="paragraph">
<p>If you are running HBase in standalone mode, you don&#8217;t need to configure anything for your client to work provided that they are all on the same machine.</p>
</div>
<div class="paragraph">
<p>Since the HBase Master may move around, clients bootstrap by looking to ZooKeeper for current critical locations.
ZooKeeper is where all these values are kept.
Thus clients require the location of the ZooKeeper ensemble before they can do anything else.
Usually this ensemble location is kept out in the <em>hbase-site.xml</em> and is picked up by the client from the <code>CLASSPATH</code>.</p>
</div>
<div class="paragraph">
<p>If you are configuring an IDE to run an HBase client, you should include the <em>conf/</em> directory on your classpath so <em>hbase-site.xml</em> settings can be found (or add <em>src/test/resources</em> to pick up the hbase-site.xml used by tests).</p>
</div>
<div class="paragraph">
<p>For Java applications using Maven, including the hbase-shaded-client module is the recommended dependency when connecting to a cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.hbase<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>hbase-shaded-client<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>2.0.0<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>A basic example <em>hbase-site.xml</em> for client only may look as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="preprocessor">&lt;?xml version=&quot;1.0&quot;?&gt;</span>
<span class="preprocessor">&lt;?xml-stylesheet type=&quot;text/xsl&quot; href=&quot;configuration.xsl&quot;?&gt;</span>
<span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.quorum<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>example1,example2,example3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>The directory shared by region servers.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
<div class="sect3">
<h4 id="java.client.config"><a class="anchor" href="#java.client.config"></a>7.5.1. Java client configuration</h4>
<div class="paragraph">
<p>The configuration used by a Java client is kept in an <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration">HBaseConfiguration</a> instance.</p>
</div>
<div class="paragraph">
<p>The factory method on HBaseConfiguration, <code>HBaseConfiguration.create();</code>, on invocation, will read in the content of the first <em>hbase-site.xml</em> found on the client&#8217;s <code>CLASSPATH</code>, if one is present (Invocation will also factor in any <em>hbase-default.xml</em> found; an <em>hbase-default.xml</em> ships inside the <em>hbase.X.X.X.jar</em>). It is also possible to specify configuration directly without having to read from a <em>hbase-site.xml</em>.
For example, to set the ZooKeeper ensemble for the cluster programmatically do as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
config.set(<span class="string"><span class="delimiter">&quot;</span><span class="content">hbase.zookeeper.quorum</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">localhost</span><span class="delimiter">&quot;</span></span>); <span class="comment">// Here we are running zookeeper locally</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>If multiple ZooKeeper instances make up your ZooKeeper ensemble, they may be specified in a comma-separated list (just as in the <em>hbase-site.xml</em> file). This populated <code>Configuration</code> instance can then be passed to an <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html">Table</a>, and so on.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="config_timeouts"><a class="anchor" href="#config_timeouts"></a>7.6. Timeout settings</h3>
<div class="paragraph">
<p>HBase provides a wide variety of timeout settings to limit the execution time of various remote operations.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>hbase.rpc.timeout</p>
</li>
<li>
<p>hbase.rpc.read.timeout</p>
</li>
<li>
<p>hbase.rpc.write.timeout</p>
</li>
<li>
<p>hbase.client.operation.timeout</p>
</li>
<li>
<p>hbase.client.meta.operation.timeout</p>
</li>
<li>
<p>hbase.client.scanner.timeout.period</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The <code>hbase.rpc.timeout</code> property limits how long a single RPC call can run before timing out.
To fine tune read or write related RPC timeouts set <code>hbase.rpc.read.timeout</code> and <code>hbase.rpc.write.timeout</code> configuration properties.
In the absence of these properties <code>hbase.rpc.timeout</code> will be used.</p>
</div>
<div class="paragraph">
<p>A higher-level timeout is <code>hbase.client.operation.timeout</code> which is valid for each client call.
When an RPC call fails for instance for a timeout due to <code>hbase.rpc.timeout</code> it will be retried until <code>hbase.client.operation.timeout</code> is reached.
Client operation timeout for system tables can be fine tuned by setting <code>hbase.client.meta.operation.timeout</code> configuration value.
When this is not set its value will use <code>hbase.client.operation.timeout</code>.</p>
</div>
<div class="paragraph">
<p>Timeout for scan operations is controlled differently. Use <code>hbase.client.scanner.timeout.period</code> property to set this timeout.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="example_config"><a class="anchor" href="#example_config"></a>8. Example Configurations</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_basic_distributed_hbase_install"><a class="anchor" href="#_basic_distributed_hbase_install"></a>8.1. Basic Distributed HBase Install</h3>
<div class="paragraph">
<p>Here is a basic configuration example for a distributed ten node cluster:
* The nodes are named <code>example0</code>, <code>example1</code>, etc., through node <code>example9</code> in this example.
* The HBase Master and the HDFS NameNode are running on the node <code>example0</code>.
* RegionServers run on nodes <code>example1</code>-<code>example9</code>.
* A 3-node ZooKeeper ensemble runs on <code>example1</code>, <code>example2</code>, and <code>example3</code> on the default ports.
* ZooKeeper data is persisted to the directory <em>/export/zookeeper</em>.</p>
</div>
<div class="paragraph">
<p>Below we show what the main configuration files&#8201;&#8212;&#8201;<em>hbase-site.xml</em>, <em>regionservers</em>, and <em>hbase-env.sh</em>&#8201;&#8212;&#8201;found in the HBase <em>conf</em> directory might look like.</p>
</div>
<div class="sect3">
<h4 id="hbase_site"><a class="anchor" href="#hbase_site"></a>8.1.1. <em>hbase-site.xml</em></h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="preprocessor">&lt;?xml version=&quot;1.0&quot;?&gt;</span>
<span class="preprocessor">&lt;?xml-stylesheet type=&quot;text/xsl&quot; href=&quot;configuration.xsl&quot;?&gt;</span>
<span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.quorum<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>example1,example2,example3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>The directory shared by RegionServers.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.dataDir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/export/zookeeper<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Property from ZooKeeper config zoo.cfg.
The directory where the snapshot is stored.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rootdir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hdfs://example0:8020/hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>The directory shared by RegionServers.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.cluster.distributed<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>The mode the cluster will be in. Possible values are
false: standalone and pseudo-distributed setups with managed ZooKeeper
true: fully-distributed with unmanaged ZooKeeper Quorum (see hbase-env.sh)
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="regionservers"><a class="anchor" href="#regionservers"></a>8.1.2. <em>regionservers</em></h4>
<div class="paragraph">
<p>In this file you list the nodes that will run RegionServers.
In our case, these nodes are <code>example1</code>-<code>example9</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">example1
example2
example3
example4
example5
example6
example7
example8
example9</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="hbase_env"><a class="anchor" href="#hbase_env"></a>8.1.3. <em>hbase-env.sh</em></h4>
<div class="paragraph">
<p>The following lines in the <em>hbase-env.sh</em> file show how to set the <code>JAVA_HOME</code> environment variable (required for HBase) and set the heap to 4 GB (rather than the default value of 1 GB). If you copy and paste this example, be sure to adjust the <code>JAVA_HOME</code> to suit your environment.</p>
</div>
<div class="listingblock">
<div class="content">
<pre># The java implementation to use.
export JAVA_HOME=/usr/java/jdk1.8.0/
# The maximum amount of heap to use. Default is left to JVM default.
export HBASE_HEAPSIZE=4G</pre>
</div>
</div>
<div class="paragraph">
<p>Use rsync to copy the content of the <em>conf</em> directory to all nodes of the cluster.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="important_configurations"><a class="anchor" href="#important_configurations"></a>9. The Important Configurations</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Below we list some <em>important</em> configurations.
We&#8217;ve divided this section into required configuration and worth-a-look recommended configs.</p>
</div>
<div class="sect2">
<h3 id="required_configuration"><a class="anchor" href="#required_configuration"></a>9.1. Required Configurations</h3>
<div class="paragraph">
<p>Review the <a href="#os">os</a> and <a href="#hadoop">hadoop</a> sections.</p>
</div>
<div class="sect3">
<h4 id="big.cluster.config"><a class="anchor" href="#big.cluster.config"></a>9.1.1. Big Cluster Configurations</h4>
<div class="paragraph">
<p>If you have a cluster with a lot of regions, it is possible that a Regionserver checks in briefly after the Master starts while all the remaining RegionServers lag behind. This first server to check in will be assigned all regions which is not optimal.
To prevent the above scenario from happening, up the <code>hbase.master.wait.on.regionservers.mintostart</code> property from its default value of 1.
See <a href="https://issues.apache.org/jira/browse/HBASE-6389">HBASE-6389 Modify the
conditions to ensure that Master waits for sufficient number of Region Servers before
starting region assignments</a> for more detail.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="recommended_configurations"><a class="anchor" href="#recommended_configurations"></a>9.2. Recommended Configurations</h3>
<div class="sect3">
<h4 id="recommended_configurations.zk"><a class="anchor" href="#recommended_configurations.zk"></a>9.2.1. ZooKeeper Configuration</h4>
<div class="sect4">
<h5 id="sect.zookeeper.session.timeout"><a class="anchor" href="#sect.zookeeper.session.timeout"></a><code>zookeeper.session.timeout</code></h5>
<div class="paragraph">
<p>The default timeout is 90 seconds (specified in milliseconds). This means that if a server crashes, it will be 90 seconds before the Master notices the crash and starts recovery.
You might need to tune the timeout down to a minute or even less so the Master notices failures sooner.
Before changing this value, be sure you have your JVM garbage collection configuration under control, otherwise, a long garbage collection that lasts beyond the ZooKeeper session timeout will take out your RegionServer. (You might be fine with this&#8201;&#8212;&#8201;you probably want recovery to start on the server if a RegionServer has been in GC for a long period of time).</p>
</div>
<div class="paragraph">
<p>To change this configuration, edit <em>hbase-site.xml</em>, copy the changed file across the cluster and restart.</p>
</div>
<div class="paragraph">
<p>We set this value high to save our having to field questions up on the mailing lists asking why a RegionServer went down during a massive import.
The usual cause is that their JVM is untuned and they are running into long GC pauses.
Our thinking is that while users are getting familiar with HBase, we&#8217;d save them having to know all of its intricacies.
Later when they&#8217;ve built some confidence, then they can play with configuration such as this.</p>
</div>
</div>
<div class="sect4">
<h5 id="zookeeper.instances"><a class="anchor" href="#zookeeper.instances"></a>Number of ZooKeeper Instances</h5>
<div class="paragraph">
<p>See <a href="#zookeeper">zookeeper</a>.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="recommended.configurations.hdfs"><a class="anchor" href="#recommended.configurations.hdfs"></a>9.2.2. HDFS Configurations</h4>
<div class="sect4">
<h5 id="dfs.datanode.failed.volumes.tolerated"><a class="anchor" href="#dfs.datanode.failed.volumes.tolerated"></a><code>dfs.datanode.failed.volumes.tolerated</code></h5>
<div class="paragraph">
<p>This is the "&#8230;&#8203;number of volumes that are allowed to fail before a DataNode stops offering service.
By default any volume failure will cause a datanode to shutdown" from the <em>hdfs-default.xml</em> description.
You might want to set this to about half the amount of your available disks.</p>
</div>
</div>
<div class="sect4">
<h5 id="hbase.regionserver.handler.count"><a class="anchor" href="#hbase.regionserver.handler.count"></a><code>hbase.regionserver.handler.count</code></h5>
<div class="paragraph">
<p>This setting defines the number of threads that are kept open to answer incoming requests to user tables.
The rule of thumb is to keep this number low when the payload per request approaches the MB (big puts, scans using a large cache) and high when the payload is small (gets, small puts, ICVs, deletes). The total size of the queries in progress is limited by the setting <code>hbase.ipc.server.max.callqueue.size</code>.</p>
</div>
<div class="paragraph">
<p>It is safe to set that number to the maximum number of incoming clients if their payload is small, the typical example being a cluster that serves a website since puts aren&#8217;t typically buffered and most of the operations are gets.</p>
</div>
<div class="paragraph">
<p>The reason why it is dangerous to keep this setting high is that the aggregate size of all the puts that are currently happening in a region server may impose too much pressure on its memory, or even trigger an OutOfMemoryError.
A RegionServer running on low memory will trigger its JVM&#8217;s garbage collector to run more frequently up to a point where GC pauses become noticeable (the reason being that all the memory used to keep all the requests' payloads cannot be trashed, no matter how hard the garbage collector tries). After some time, the overall cluster throughput is affected since every request that hits that RegionServer will take longer, which exacerbates the problem even more.</p>
</div>
<div class="paragraph">
<p>You can get a sense of whether you have too little or too many handlers by <a href="#rpc.logging">rpc.logging</a> on an individual RegionServer then tailing its logs (Queued requests consume memory).</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="big_memory"><a class="anchor" href="#big_memory"></a>9.2.3. Configuration for large memory machines</h4>
<div class="paragraph">
<p>HBase ships with a reasonable, conservative configuration that will work on nearly all machine types that people might want to test with.
If you have larger machines&#8201;&#8212;&#8201;HBase has 8G and larger heap&#8201;&#8212;&#8201;you might find the following configuration options helpful.
TODO.</p>
</div>
</div>
<div class="sect3">
<h4 id="config.compression"><a class="anchor" href="#config.compression"></a>9.2.4. Compression</h4>
<div class="paragraph">
<p>You should consider enabling ColumnFamily compression.
There are several options that are near-frictionless and in most all cases boost performance by reducing the size of StoreFiles and thus reducing I/O.</p>
</div>
<div class="paragraph">
<p>See <a href="#compression">compression</a> for more information.</p>
</div>
</div>
<div class="sect3">
<h4 id="config.wals"><a class="anchor" href="#config.wals"></a>9.2.5. Configuring the size and number of WAL files</h4>
<div class="paragraph">
<p>HBase uses <a href="#wal">wal</a> to recover the memstore data that has not been flushed to disk in case of an RS failure.
These WAL files should be configured to be slightly smaller than HDFS block (by default a HDFS block is 64Mb and a WAL file is ~60Mb).</p>
</div>
<div class="paragraph">
<p>HBase also has a limit on the number of WAL files, designed to ensure there&#8217;s never too much data that needs to be replayed during recovery.
This limit needs to be set according to memstore configuration, so that all the necessary data would fit.
It is recommended to allocate enough WAL files to store at least that much data (when all memstores are close to full). For example, with 16Gb RS heap, default memstore settings (0.4), and default WAL file size (~60Mb), 16Gb*0.4/60, the starting point for WAL file count is ~109.
However, as all memstores are not expected to be full all the time, less WAL files can be allocated.</p>
</div>
</div>
<div class="sect3">
<h4 id="disable.splitting"><a class="anchor" href="#disable.splitting"></a>9.2.6. Managed Splitting</h4>
<div class="paragraph">
<p>HBase generally handles splitting of your regions based upon the settings in your <em>hbase-default.xml</em> and <em>hbase-site.xml</em> configuration files.
Important settings include <code>hbase.regionserver.region.split.policy</code>, <code>hbase.hregion.max.filesize</code>, <code>hbase.regionserver.regionSplitLimit</code>.
A simplistic view of splitting is that when a region grows to <code>hbase.hregion.max.filesize</code>, it is split.
For most usage patterns, you should use automatic splitting.
See <a href="#manual_region_splitting_decisions">manual region splitting decisions</a> for more information about manual region splitting.</p>
</div>
<div class="paragraph">
<p>Instead of allowing HBase to split your regions automatically, you can choose to manage the splitting yourself.
Manually managing splits works if you know your keyspace well, otherwise let HBase figure where to split for you.
Manual splitting can mitigate region creation and movement under load.
It also makes it so region boundaries are known and invariant (if you disable region splitting). If you use manual splits, it is easier doing staggered, time-based major compactions to spread out your network IO load.</p>
</div>
<div class="paragraph">
<div class="title">Disable Automatic Splitting</div>
<p>To disable automatic splitting, you can set region split policy in either cluster configuration or table configuration to be <code>org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy</code></p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Automatic Splitting Is Recommended</div>
<div class="paragraph">
<p>If you disable automatic splits to diagnose a problem or during a period of fast data growth, it is recommended to re-enable them when your situation becomes more stable.
The potential benefits of managing region splits yourself are not undisputed.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<div class="title">Determine the Optimal Number of Pre-Split Regions</div>
<p>The optimal number of pre-split regions depends on your application and environment.
A good rule of thumb is to start with 10 pre-split regions per server and watch as data grows over time.
It is better to err on the side of too few regions and perform rolling splits later.
The optimal number of regions depends upon the largest StoreFile in your region.
The size of the largest StoreFile will increase with time if the amount of data grows.
The goal is for the largest region to be just large enough that the compaction selection algorithm only compacts it during a timed major compaction.
Otherwise, the cluster can be prone to compaction storms with a large number of regions under compaction at the same time.
It is important to understand that the data growth causes compaction storms and not the manual split decision.</p>
</div>
<div class="paragraph">
<p>If the regions are split into too many large regions, you can increase the major compaction interval by configuring <code>HConstants.MAJOR_COMPACTION_PERIOD</code>.
The <code>org.apache.hadoop.hbase.util.RegionSplitter</code> utility also provides a network-IO-safe rolling split of all regions.</p>
</div>
</div>
<div class="sect3">
<h4 id="managed.compactions"><a class="anchor" href="#managed.compactions"></a>9.2.7. Managed Compactions</h4>
<div class="paragraph">
<p>By default, major compactions are scheduled to run once in a 7-day period.</p>
</div>
<div class="paragraph">
<p>If you need to control exactly when and how often major compaction runs, you can disable managed major compactions.
See the entry for <code>hbase.hregion.majorcompaction</code> in the <a href="#compaction.parameters">compaction.parameters</a> table for details.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Do Not Disable Major Compactions</div>
<div class="paragraph">
<p>Major compactions are absolutely necessary for StoreFile clean-up.
Do not disable them altogether.
You can run major compactions manually via the HBase shell or via the <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact-org.apache.hadoop.hbase.TableName-">Admin API</a>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>For more information about compactions and the compaction file selection process, see <a href="#compaction">compaction</a></p>
</div>
</div>
<div class="sect3">
<h4 id="spec.ex"><a class="anchor" href="#spec.ex"></a>9.2.8. Speculative Execution</h4>
<div class="paragraph">
<p>Speculative Execution of MapReduce tasks is on by default, and for HBase clusters it is generally advised to turn off Speculative Execution at a system-level unless you need it for a specific case, where it can be configured per-job.
Set the properties <code>mapreduce.map.speculative</code> and <code>mapreduce.reduce.speculative</code> to false.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="other_configuration"><a class="anchor" href="#other_configuration"></a>9.3. Other Configurations</h3>
<div class="sect3">
<h4 id="balancer_config"><a class="anchor" href="#balancer_config"></a>9.3.1. Balancer</h4>
<div class="paragraph">
<p>The balancer is a periodic operation which is run on the master to redistribute regions on the cluster.
It is configured via <code>hbase.balancer.period</code> and defaults to 300000 (5 minutes).</p>
</div>
<div class="paragraph">
<p>See <a href="#master.processes.loadbalancer">master.processes.loadbalancer</a> for more information on the LoadBalancer.</p>
</div>
</div>
<div class="sect3">
<h4 id="disabling.blockcache"><a class="anchor" href="#disabling.blockcache"></a>9.3.2. Disabling Blockcache</h4>
<div class="paragraph">
<p>Do not turn off block cache (You&#8217;d do it by setting <code>hfile.block.cache.size</code> to zero). Currently we do not do well if you do this because the RegionServer will spend all its time loading HFile indices over and over again.
If your working set is such that block cache does you no good, at least size the block cache such that HFile indices will stay up in the cache (you can get a rough idea on the size you need by surveying RegionServer UIs; you&#8217;ll see index block size accounted near the top of the webpage).</p>
</div>
</div>
<div class="sect3">
<h4 id="nagles"><a class="anchor" href="#nagles"></a>9.3.3. <a href="http://en.wikipedia.org/wiki/Nagle&#8217;s_algorithm">Nagle&#8217;s</a> or the small package problem</h4>
<div class="paragraph">
<p>If a big 40ms or so occasional delay is seen in operations against HBase, try the Nagles' setting.
For example, see the user mailing list thread, <a href="http://search-hadoop.com/m/pduLg2fydtE/Inconsistent+scan+performance+with+caching+set+&amp;subj=Re+Inconsistent+scan+performance+with+caching+set+to+1">Inconsistent scan performance with caching set to 1</a> and the issue cited therein where setting <code>notcpdelay</code> improved scan speeds.
You might also see the graphs on the tail of <a href="https://issues.apache.org/jira/browse/HBASE-7008">HBASE-7008 Set scanner caching to a better default</a> where our Lars Hofhansl tries various data sizes w/ Nagle&#8217;s on and off measuring the effect.</p>
</div>
</div>
<div class="sect3">
<h4 id="mttr"><a class="anchor" href="#mttr"></a>9.3.4. Better Mean Time to Recover (MTTR)</h4>
<div class="paragraph">
<p>This section is about configurations that will make servers come back faster after a fail.
See the Deveraj Das and Nicolas Liochon blog post <a href="http://hortonworks.com/blog/introduction-to-hbase-mean-time-to-recover-mttr/">Introduction to HBase Mean Time to Recover (MTTR)</a> for a brief introduction.</p>
</div>
<div class="paragraph">
<p>The issue <a href="https://issues.apache.org/jira/browse/HBASE-8389">HBASE-8354 forces Namenode into loop with lease recovery requests</a> is messy but has a bunch of good discussion toward the end on low timeouts and how to cause faster recovery including citation of fixes added to HDFS. Read the Varun Sharma comments.
The below suggested configurations are Varun&#8217;s suggestions distilled and tested.
Make sure you are running on a late-version HDFS so you have the fixes he refers to and himself adds to HDFS that help HBase MTTR (e.g.
HDFS-3703, HDFS-3712, and HDFS-4791&#8201;&#8212;&#8201;Hadoop 2 for sure has them and late Hadoop 1 has some). Set the following in the RegionServer.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.lease.recovery.dfs.timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>23000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>How much time we allow elapse between calls to recover lease.
Should be larger than the dfs timeout.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.client.socket-timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down the DFS timeout from 60 to 10 seconds.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>And on the NameNode/DataNode side, set the following to enable 'staleness' introduced in HDFS-3703, HDFS-3912.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.client.socket-timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down the DFS timeout from 60 to 10 seconds.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.datanode.socket.write.timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down the DFS timeout from 8 * 60 to 10 seconds.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>ipc.client.connect.timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down from 60 seconds to 3.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>ipc.client.connect.max.retries.on.timeouts<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>2<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down from 45 seconds to 3 (2 == 3 retries).<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.namenode.avoid.read.stale.datanode<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Enable stale state in hdfs<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.namenode.stale.datanode.interval<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>20000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Down from default 30 seconds<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.namenode.avoid.write.stale.datanode<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Enable stale state in hdfs<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="jmx_config"><a class="anchor" href="#jmx_config"></a>9.3.5. JMX</h4>
<div class="paragraph">
<p>JMX (Java Management Extensions) provides built-in instrumentation that enables you to monitor and manage the Java VM.
To enable monitoring and management from remote systems, you need to set system property <code>com.sun.management.jmxremote.port</code> (the port number through which you want to enable JMX RMI connections) when you start the Java VM.
See the <a href="http://docs.oracle.com/javase/8/docs/technotes/guides/management/agent.html">official documentation</a> for more information.
Historically, besides above port mentioned, JMX opens two additional random TCP listening ports, which could lead to port conflict problem. (See <a href="https://issues.apache.org/jira/browse/HBASE-10289">HBASE-10289</a> for details)</p>
</div>
<div class="paragraph">
<p>As an alternative, you can use the coprocessor-based JMX implementation provided by HBase.
To enable it, add below property in <em>hbase-site.xml</em>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.regionserver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.JMXListener<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
DO NOT set <code>com.sun.management.jmxremote.port</code> for Java VM at the same time.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Currently it supports Master and RegionServer Java VM.
By default, the JMX listens on TCP port 10102, you can further configure the port using below properties:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>regionserver.rmi.registry.port<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>61130<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>regionserver.rmi.connector.port<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>61140<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The registry port can be shared with connector port in most cases, so you only need to configure regionserver.rmi.registry.port.
However if you want to use SSL communication, the 2 ports must be configured to different values.</p>
</div>
<div class="paragraph">
<p>By default the password authentication and SSL communication is disabled.
To enable password authentication, you need to update <em>hbase-env.sh</em> like below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">export HBASE_JMX_BASE=&quot;-Dcom.sun.management.jmxremote.authenticate=true \
-Dcom.sun.management.jmxremote.password.file=your_password_file \
-Dcom.sun.management.jmxremote.access.file=your_access_file&quot;
export HBASE_MASTER_OPTS=&quot;$HBASE_MASTER_OPTS $HBASE_JMX_BASE &quot;
export HBASE_REGIONSERVER_OPTS=&quot;$HBASE_REGIONSERVER_OPTS $HBASE_JMX_BASE &quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>See example password/access file under <em>$JRE_HOME/lib/management</em>.</p>
</div>
<div class="paragraph">
<p>To enable SSL communication with password authentication, follow below steps:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">#1. generate a key pair, stored in myKeyStore
keytool -genkey -alias jconsole -keystore myKeyStore
#2. export it to file jconsole.cert
keytool -export -alias jconsole -keystore myKeyStore -file jconsole.cert
#3. copy jconsole.cert to jconsole client machine, import it to jconsoleKeyStore
keytool -import -alias jconsole -keystore jconsoleKeyStore -file jconsole.cert</code></pre>
</div>
</div>
<div class="paragraph">
<p>And then update <em>hbase-env.sh</em> like below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">export HBASE_JMX_BASE=&quot;-Dcom.sun.management.jmxremote.ssl=true \
-Djavax.net.ssl.keyStore=/home/tianq/myKeyStore \
-Djavax.net.ssl.keyStorePassword=your_password_in_step_1 \
-Dcom.sun.management.jmxremote.authenticate=true \
-Dcom.sun.management.jmxremote.password.file=your_password file \
-Dcom.sun.management.jmxremote.access.file=your_access_file&quot;
export HBASE_MASTER_OPTS=&quot;$HBASE_MASTER_OPTS $HBASE_JMX_BASE &quot;
export HBASE_REGIONSERVER_OPTS=&quot;$HBASE_REGIONSERVER_OPTS $HBASE_JMX_BASE &quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Finally start <code>jconsole</code> on the client using the key store:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">jconsole -J-Djavax.net.ssl.trustStore=/home/tianq/jconsoleKeyStore</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
To enable the HBase JMX implementation on Master, you also need to add below property in <em>hbase-site.xml</em>:
</td>
</tr>
</table>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.JMXListener<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The corresponding properties for port configuration are <code>master.rmi.registry.port</code> (by default 10101) and <code>master.rmi.connector.port</code> (by default the same as registry.port)</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="dyn_config"><a class="anchor" href="#dyn_config"></a>10. Dynamic Configuration</h2>
<div class="sectionbody">
<div class="paragraph">
<p>It is possible to change a subset of the configuration without requiring a server restart.
In the HBase shell, the operations <code>update_config</code> and <code>update_all_config</code> will prompt a server or all servers to reload configuration.</p>
</div>
<div class="paragraph">
<p>Only a subset of all configurations can currently be changed in the running server.
Here are those configurations:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 3. Configurations support dynamically change</caption>
<colgroup>
<col style="width: 100%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Key</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.fallback-to-simple-auth-allowed</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.cleaner.scan.dir.concurrent.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.thread.compaction.large</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.thread.compaction.small</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.thread.split</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.throughput.controller</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.thread.hfilecleaner.throttle</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.large.queue.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.small.queue.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.large.thread.count</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.small.thread.count</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.thread.timeout.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.hfilecleaner.thread.check.interval.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.flush.throughput.controller</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.max.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.max.size.offpeak</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.min.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.min</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.max</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.ratio</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.ratio.offpeak</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regionserver.thread.compaction.throttle</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hregion.majorcompaction</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hregion.majorcompaction.jitter</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.min.locality.to.skip.major.compact</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.date.tiered.max.storefile.age.millis</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.date.tiered.incoming.window.min</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.date.tiered.window.policy.class</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.date.tiered.single.output.for.minor.compaction</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hstore.compaction.date.tiered.window.factory.class</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.offpeak.start.hour</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.offpeak.end.hour</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.oldwals.cleaner.thread.size</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.oldwals.cleaner.thread.timeout.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.oldwals.cleaner.thread.check.interval.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.procedure.worker.keep.alive.time.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.procedure.worker.add.stuck.percentage</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.procedure.worker.monitor.interval.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.procedure.worker.stuck.threshold.msec</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regions.slop</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.regions.overallSlop</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.balancer.tablesOnMaster</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.balancer.tablesOnMaster.systemTablesOnly</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.util.ip.to.rack.determiner</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.max.callqueue.length</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.priority.max.callqueue.length</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.callqueue.type</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.callqueue.codel.target.delay</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.callqueue.codel.interval</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.ipc.server.callqueue.codel.lifo.threshold</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.maxSteps</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.stepsPerRegion</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.maxRunningTime</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.runMaxSteps</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.numRegionLoadsToRemember</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.loadbalance.bytable</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.minCostNeedBalance</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.localityCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.rackLocalityCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.readRequestCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.writeRequestCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.memstoreSizeCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.storefileSizeCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.regionReplicaHostCostKey</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.regionReplicaRackCostKey</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.regionCountCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.primaryRegionCountCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.moveCost</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.maxMovePercent</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.master.balancer.stochastic.tableSkewCost</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<h1 id="upgrading" class="sect0"><a class="anchor" href="#upgrading"></a>Upgrading</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>You cannot skip major versions when upgrading. If you are upgrading from version 0.98.x to 2.x, you must first go from 0.98.x to 1.2.x and then go from 1.2.x to 2.x.</p>
</div>
<div class="paragraph">
<p>Review <a href="#configuration">Apache HBase Configuration</a>, in particular <a href="#hadoop"><a href="https://hadoop.apache.org">Hadoop</a></a>. Familiarize yourself with <a href="#hbase_supported_tested_definitions">Support and Testing Expectations</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.versioning"><a class="anchor" href="#hbase.versioning"></a>11. HBase version number and compatibility</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="hbase.versioning.post10"><a class="anchor" href="#hbase.versioning.post10"></a>11.1. Aspirational Semantic Versioning</h3>
<div class="paragraph">
<p>Starting with the 1.0.0 release, HBase is working towards <a href="http://semver.org/">Semantic Versioning</a> for its release versioning. In summary:</p>
</div>
<div class="ulist">
<div class="title">Given a version number MAJOR.MINOR.PATCH, increment the:</div>
<ul>
<li>
<p>MAJOR version when you make incompatible API changes,</p>
</li>
<li>
<p>MINOR version when you add functionality in a backwards-compatible manner, and</p>
</li>
<li>
<p>PATCH version when you make backwards-compatible bug fixes.</p>
</li>
<li>
<p>Additional labels for pre-release and build metadata are available as extensions to the MAJOR.MINOR.PATCH format.</p>
</li>
</ul>
</div>
<div id="hbase.versioning.compat" class="paragraph">
<div class="title">Compatibility Dimensions</div>
<p>In addition to the usual API versioning considerations HBase has other compatibility dimensions that we need to consider.</p>
</div>
<div class="ulist">
<div class="title">Client-Server wire protocol compatibility</div>
<ul>
<li>
<p>Allows updating client and server out of sync.</p>
</li>
<li>
<p>We could only allow upgrading the server first. I.e. the server would be backward compatible to an old client, that way new APIs are OK.</p>
</li>
<li>
<p>Example: A user should be able to use an old client to connect to an upgraded cluster.</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Server-Server protocol compatibility</div>
<ul>
<li>
<p>Servers of different versions can co-exist in the same cluster.</p>
</li>
<li>
<p>The wire protocol between servers is compatible.</p>
</li>
<li>
<p>Workers for distributed tasks, such as replication and log splitting, can co-exist in the same cluster.</p>
</li>
<li>
<p>Dependent protocols (such as using ZK for coordination) will also not be changed.</p>
</li>
<li>
<p>Example: A user can perform a rolling upgrade.</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">File format compatibility</div>
<ul>
<li>
<p>Support file formats backward and forward compatible</p>
</li>
<li>
<p>Example: File, ZK encoding, directory layout is upgraded automatically as part of an HBase upgrade. User can downgrade to the older version and everything will continue to work.</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Client API compatibility</div>
<ul>
<li>
<p>Allow changing or removing existing client APIs.</p>
</li>
<li>
<p>An API needs to be deprecated for a whole major version before we will change/remove it.</p>
<div class="ulist">
<ul>
<li>
<p>An example: An API was deprecated in 2.0.1 and will be marked for deletion in 4.0.0. On the other hand, an API deprecated in 2.0.0 can be removed in 3.0.0.</p>
</li>
</ul>
</div>
</li>
<li>
<p>APIs available in a patch version will be available in all later patch versions. However, new APIs may be added which will not be available in earlier patch versions.</p>
</li>
<li>
<p>New APIs introduced in a patch version will only be added in a source compatible way <sup class="footnote">[<a id="_footnoteref_1" class="footnote" href="#_footnote_1" title="View footnote.">1</a>]</sup>: i.e. code that implements public APIs will continue to compile.</p>
<div class="ulist">
<ul>
<li>
<p>Example: A user using a newly deprecated API does not need to modify application code with HBase API calls until the next major version.
*</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Client Binary compatibility</div>
<ul>
<li>
<p>Client code written to APIs available in a given patch release can run unchanged (no recompilation needed) against the new jars of later patch versions.</p>
</li>
<li>
<p>Client code written to APIs available in a given patch release might not run against the old jars from an earlier patch version.</p>
<div class="ulist">
<ul>
<li>
<p>Example: Old compiled client code will work unchanged with the new jars.</p>
</li>
</ul>
</div>
</li>
<li>
<p>If a Client implements an HBase Interface, a recompile MAY be required upgrading to a newer minor version (See release notes
for warning about incompatible changes). All effort will be made to provide a default implementation so this case should not arise.</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Server-Side Limited API compatibility (taken from Hadoop)</div>
<ul>
<li>
<p>Internal APIs are marked as Stable, Evolving, or Unstable</p>
</li>
<li>
<p>This implies binary compatibility for coprocessors and plugins (pluggable classes, including replication) as long as these are only using marked interfaces/classes.</p>
</li>
<li>
<p>Example: Old compiled Coprocessor, Filter, or Plugin code will work unchanged with the new jars.</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Dependency Compatibility</div>
<ul>
<li>
<p>An upgrade of HBase will not require an incompatible upgrade of a dependent project, except for Apache Hadoop.</p>
</li>
<li>
<p>An upgrade of HBase will not require an incompatible upgrade of the Java runtime.</p>
</li>
<li>
<p>Example: Upgrading HBase to a version that supports <em>Dependency Compatibility</em> won&#8217;t require that you upgrade your Apache ZooKeeper service.</p>
</li>
<li>
<p>Example: If your current version of HBase supported running on JDK 8, then an upgrade to a version that supports <em>Dependency Compatibility</em> will also run on JDK 8.</p>
</li>
</ul>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="title">Hadoop Versions</div>
<div class="paragraph">
<p>Previously, we tried to maintain dependency compatibility for the underly Hadoop service but over the last few years this has proven untenable. While the HBase project attempts to maintain support for older versions of Hadoop, we drop the "supported" designator for minor versions that fail to continue to see releases. Additionally, the Hadoop project has its own set of compatibility guidelines, which means in some cases having to update to a newer supported minor release might break some of our compatibility promises.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="ulist">
<div class="title">Operational Compatibility</div>
<ul>
<li>
<p>Metric changes</p>
</li>
<li>
<p>Behavioral changes of services</p>
</li>
<li>
<p>JMX APIs exposed via the <code>/jmx/</code> endpoint</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Summary</div>
<ul>
<li>
<p>A patch upgrade is a drop-in replacement. Any change that is not Java binary and source compatible would not be allowed.<sup class="footnote">[<a id="_footnoteref_2" class="footnote" href="#_footnote_2" title="View footnote.">2</a>]</sup> Downgrading versions within patch releases may not be compatible.</p>
</li>
<li>
<p>A minor upgrade requires no application/client code modification. Ideally it would be a drop-in replacement but client code, coprocessors, filters, etc might have to be recompiled if new jars are used.</p>
</li>
<li>
<p>A major upgrade allows the HBase community to make breaking changes.</p>
</li>
</ul>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 4. Compatibility Matrix <sup class="footnote">[<a id="_footnoteref_3" class="footnote" href="#_footnote_3" title="View footnote.">3</a>]</sup></caption>
<colgroup>
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Major</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Minor</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Patch</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Client-Server wire Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Server-Server Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">File Format Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N <sup class="footnote">[<a id="_footnoteref_4" class="footnote" href="#_footnote_4" title="View footnote.">4</a>]</sup></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Client API Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Client Binary Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top" colspan="4"><p class="tableblock">Server-Side Limited API Compatibility</p></td>
</tr>
<tr>
<td class="tableblock halign-right valign-top"><p class="tableblock">Stable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-right valign-top"><p class="tableblock">Evolving</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-right valign-top"><p class="tableblock">Unstable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Dependency Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Operational Compatibility</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">N</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Y</p></td>
</tr>
</tbody>
</table>
<div class="sect3">
<h4 id="hbase.client.api.surface"><a class="anchor" href="#hbase.client.api.surface"></a>11.1.1. HBase API Surface</h4>
<div class="paragraph">
<p>HBase has a lot of API points, but for the compatibility matrix above, we differentiate between Client API, Limited Private API, and Private API. HBase uses <a href="https://yetus.apache.org/documentation/0.5.0/interface-classification/">Apache Yetus Audience Annotations</a> to guide downstream expectations for stability.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>InterfaceAudience (<a href="https://yetus.apache.org/documentation/0.5.0/audience-annotations-apidocs/org/apache/yetus/audience/InterfaceAudience.html">javadocs</a>): captures the intended audience, possible values include:</p>
<div class="ulist">
<ul>
<li>
<p>Public: safe for end users and external projects</p>
</li>
<li>
<p>LimitedPrivate: used for internals we expect to be pluggable, such as coprocessors</p>
</li>
<li>
<p>Private: strictly for use within HBase itself
Classes which are defined as <code>IA.Private</code> may be used as parameters or return values for interfaces which are declared <code>IA.LimitedPrivate</code>. Treat the <code>IA.Private</code> object as opaque; do not try to access its methods or fields directly.</p>
</li>
</ul>
</div>
</li>
<li>
<p>InterfaceStability (<a href="https://yetus.apache.org/documentation/0.5.0/audience-annotations-apidocs/org/apache/yetus/audience/InterfaceStability.html">javadocs</a>): describes what types of interface changes are permitted. Possible values include:</p>
<div class="ulist">
<ul>
<li>
<p>Stable: the interface is fixed and is not expected to change</p>
</li>
<li>
<p>Evolving: the interface may change in future minor verisons</p>
</li>
<li>
<p>Unstable: the interface may change at any time</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
<div class="paragraph">
<p>Please keep in mind the following interactions between the <code>InterfaceAudience</code> and <code>InterfaceStability</code> annotations within the HBase project:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>IA.Public</code> classes are inherently stable and adhere to our stability guarantees relating to the type of upgrade (major, minor, or patch).</p>
</li>
<li>
<p><code>IA.LimitedPrivate</code> classes should always be annotated with one of the given <code>InterfaceStability</code> values. If they are not, you should presume they are <code>IS.Unstable</code>.</p>
</li>
<li>
<p><code>IA.Private</code> classes should be considered implicitly unstable, with no guarantee of stability between releases.</p>
</li>
</ul>
</div>
<div id="hbase.client.api" class="dlist">
<dl>
<dt class="hdlist1">HBase Client API</dt>
<dd>
<p>HBase Client API consists of all the classes or methods that are marked with InterfaceAudience.Public interface. All main classes in hbase-client and dependent modules have either InterfaceAudience.Public, InterfaceAudience.LimitedPrivate, or InterfaceAudience.Private marker. Not all classes in other modules (hbase-server, etc) have the marker. If a class is not annotated with one of these, it is assumed to be a InterfaceAudience.Private class.</p>
</dd>
</dl>
</div>
<div id="hbase.limitetprivate.api" class="dlist">
<dl>
<dt class="hdlist1">HBase LimitedPrivate API</dt>
<dd>
<p>LimitedPrivate annotation comes with a set of target consumers for the interfaces. Those consumers are coprocessors, phoenix, replication endpoint implementations or similar. At this point, HBase only guarantees source and binary compatibility for these interfaces between patch versions.</p>
</dd>
</dl>
</div>
<div id="hbase.private.api" class="dlist">
<dl>
<dt class="hdlist1">HBase Private API</dt>
<dd>
<p>All classes annotated with InterfaceAudience.Private or all classes that do not have the annotation are for HBase internal use only. The interfaces and method signatures can change at any point in time. If you are relying on a particular interface that is marked Private, you should open a jira to propose changing the interface to be Public or LimitedPrivate, or an interface exposed for this purpose.</p>
</dd>
</dl>
</div>
<div id="hbase.binary.compatibility" class="paragraph">
<div class="title">Binary Compatibility</div>
<p>When we say two HBase versions are compatible, we mean that the versions are wire and binary compatible. Compatible HBase versions means that clients can talk to compatible but differently versioned servers. It means too that you can just swap out the jars of one version and replace them with the jars of another, compatible version and all will just work. Unless otherwise specified, HBase point versions are (mostly) binary compatible. You can safely do rolling upgrades between binary compatible versions; i.e. across maintenance releases: e.g. from 1.4.4 to 1.4.6. See link:[Does compatibility between versions also mean binary compatibility?] discussion on the HBase dev mailing list.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.rolling.upgrade"><a class="anchor" href="#hbase.rolling.upgrade"></a>11.2. Rolling Upgrades</h3>
<div class="paragraph">
<p>A rolling upgrade is the process by which you update the servers in your cluster a server at a time. You can rolling upgrade across HBase versions if they are binary or wire compatible. See <a href="#hbase.rolling.restart">Rolling Upgrade Between Versions that are Binary/Wire Compatible</a> for more on what this means. Coarsely, a rolling upgrade is a graceful stop each server, update the software, and then restart. You do this for each server in the cluster. Usually you upgrade the Master first and then the RegionServers. See <a href="#rolling">Rolling Restart</a> for tools that can help use the rolling upgrade process.</p>
</div>
<div class="paragraph">
<p>For example, in the below, HBase was symlinked to the actual HBase install. On upgrade, before running a rolling restart over the cluster, we changed the symlink to point at the new HBase software version and then ran</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_HOME=~/hadoop-2.6.0-CRC-SNAPSHOT ~/hbase/bin/rolling-restart.sh --config ~/conf_hbase</code></pre>
</div>
</div>
<div class="paragraph">
<p>The rolling-restart script will first gracefully stop and restart the master, and then each of the RegionServers in turn. Because the symlink was changed, on restart the server will come up using the new HBase version. Check logs for errors as the rolling upgrade proceeds.</p>
</div>
<div id="hbase.rolling.restart" class="paragraph">
<div class="title">Rolling Upgrade Between Versions that are Binary/Wire Compatible</div>
<p>Unless otherwise specified, HBase minor versions are binary compatible. You can do a <a href="#hbase.rolling.upgrade">Rolling Upgrades</a> between HBase point versions. For example, you can go to 1.4.4 from 1.4.6 by doing a rolling upgrade across the cluster replacing the 1.4.4 binary with a 1.4.6 binary.</p>
</div>
<div class="paragraph">
<p>In the minor version-particular sections below, we call out where the versions are wire/protocol compatible and in this case, it is also possible to do a <a href="#hbase.rolling.upgrade">Rolling Upgrades</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_rollback"><a class="anchor" href="#_rollback"></a>12. Rollback</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Sometimes things don&#8217;t go as planned when attempting an upgrade. This section explains how to perform a <em>rollback</em> to an earlier HBase release. Note that this should only be needed between Major and some Minor releases. You should always be able to <em>downgrade</em> between HBase Patch releases within the same Minor version. These instructions may require you to take steps before you start the upgrade process, so be sure to read through this section beforehand.</p>
</div>
<div class="sect2">
<h3 id="_caveats"><a class="anchor" href="#_caveats"></a>12.1. Caveats</h3>
<div class="paragraph">
<div class="title">Rollback vs Downgrade</div>
<p>This section describes how to perform a <em>rollback</em> on an upgrade between HBase minor and major versions. In this document, rollback refers to the process of taking an upgraded cluster and restoring it to the old version <em>while losing all changes that have occurred since upgrade</em>. By contrast, a cluster <em>downgrade</em> would restore an upgraded cluster to the old version while maintaining any data written since the upgrade. We currently only offer instructions to rollback HBase clusters. Further, rollback only works when these instructions are followed prior to performing the upgrade.</p>
</div>
<div class="paragraph">
<p>When these instructions talk about rollback vs downgrade of prerequisite cluster services (i.e. HDFS), you should treat leaving the service version the same as a degenerate case of downgrade.</p>
</div>
<div class="paragraph">
<div class="title">Replication</div>
<p>Unless you are doing an all-service rollback, the HBase cluster will lose any configured peers for HBase replication. If your cluster is configured for HBase replication, then prior to following these instructions you should document all replication peers. After performing the rollback you should then add each documented peer back to the cluster. For more information on enabling HBase replication, listing peers, and adding a peer see <a href="#hbase.replication.management">Managing and Configuring Cluster Replication</a>. Note also that data written to the cluster since the upgrade may or may not have already been replicated to any peers. Determining which, if any, peers have seen replication data as well as rolling back the data in those peers is out of the scope of this guide.</p>
</div>
<div class="paragraph">
<div class="title">Data Locality</div>
<p>Unless you are doing an all-service rollback, going through a rollback procedure will likely destroy all locality for Region Servers. You should expect degraded performance until after the cluster has had time to go through compactions to restore data locality. Optionally, you can force a compaction to speed this process up at the cost of generating cluster load.</p>
</div>
<div class="paragraph">
<div class="title">Configurable Locations</div>
<p>The instructions below assume default locations for the HBase data directory and the HBase znode. Both of these locations are configurable and you should verify the value used in your cluster before proceeding. In the event that you have a different value, just replace the default with the one found in your configuration
* HBase data directory is configured via the key 'hbase.rootdir' and has a default value of '/hbase'.
* HBase znode is configured via the key 'zookeeper.znode.parent' and has a default value of '/hbase'.</p>
</div>
</div>
<div class="sect2">
<h3 id="_all_service_rollback"><a class="anchor" href="#_all_service_rollback"></a>12.2. All service rollback</h3>
<div class="paragraph">
<p>If you will be performing a rollback of both the HDFS and ZooKeeper services, then HBase&#8217;s data will be rolled back in the process.</p>
</div>
<div class="ulist">
<div class="title">Requirements</div>
<ul>
<li>
<p>Ability to rollback HDFS and ZooKeeper</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Before upgrade</div>
<p>No additional steps are needed pre-upgrade. As an extra precautionary measure, you may wish to use distcp to back up the HBase data off of the cluster to be upgraded. To do so, follow the steps in the 'Before upgrade' section of 'Rollback after HDFS downgrade' but copy to another HDFS instance instead of within the same instance.</p>
</div>
<div class="olist arabic">
<div class="title">Performing a rollback</div>
<ol class="arabic">
<li>
<p>Stop HBase</p>
</li>
<li>
<p>Perform a rollback for HDFS and ZooKeeper (HBase should remain stopped)</p>
</li>
<li>
<p>Change the installed version of HBase to the previous version</p>
</li>
<li>
<p>Start HBase</p>
</li>
<li>
<p>Verify HBase contents—use the HBase shell to list tables and scan some known values.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_rollback_after_hdfs_rollback_and_zookeeper_downgrade"><a class="anchor" href="#_rollback_after_hdfs_rollback_and_zookeeper_downgrade"></a>12.3. Rollback after HDFS rollback and ZooKeeper downgrade</h3>
<div class="paragraph">
<p>If you will be rolling back HDFS but going through a ZooKeeper downgrade, then HBase will be in an inconsistent state. You must ensure the cluster is not started until you complete this process.</p>
</div>
<div class="ulist">
<div class="title">Requirements</div>
<ul>
<li>
<p>Ability to rollback HDFS</p>
</li>
<li>
<p>Ability to downgrade ZooKeeper</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Before upgrade</div>
<p>No additional steps are needed pre-upgrade. As an extra precautionary measure, you may wish to use distcp to back up the HBase data off of the cluster to be upgraded. To do so, follow the steps in the 'Before upgrade' section of 'Rollback after HDFS downgrade' but copy to another HDFS instance instead of within the same instance.</p>
</div>
<div class="olist arabic">
<div class="title">Performing a rollback</div>
<ol class="arabic">
<li>
<p>Stop HBase</p>
</li>
<li>
<p>Perform a rollback for HDFS and a downgrade for ZooKeeper (HBase should remain stopped)</p>
</li>
<li>
<p>Change the installed version of HBase to the previous version</p>
</li>
<li>
<p>Clean out ZooKeeper information related to HBase. WARNING: This step will permanently destroy all replication peers. Please see the section on HBase Replication under Caveats for more information.</p>
<div class="listingblock">
<div class="title">Clean HBase information out of ZooKeeper</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">[hpnewton@gateway_node.example.com ~]$ zookeeper-client -server zookeeper1.example.com:2181,zookeeper2.example.com:2181,zookeeper3.example.com:2181
Welcome to ZooKeeper!
JLine support is disabled
rmr /hbase
quit
Quitting...</code></pre>
</div>
</div>
</li>
<li>
<p>Start HBase</p>
</li>
<li>
<p>Verify HBase contents—use the HBase shell to list tables and scan some known values.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_rollback_after_hdfs_downgrade"><a class="anchor" href="#_rollback_after_hdfs_downgrade"></a>12.4. Rollback after HDFS downgrade</h3>
<div class="paragraph">
<p>If you will be performing an HDFS downgrade, then you&#8217;ll need to follow these instructions regardless of whether ZooKeeper goes through rollback, downgrade, or reinstallation.</p>
</div>
<div class="ulist">
<div class="title">Requirements</div>
<ul>
<li>
<p>Ability to downgrade HDFS</p>
</li>
<li>
<p>Pre-upgrade cluster must be able to run MapReduce jobs</p>
</li>
<li>
<p>HDFS super user access</p>
</li>
<li>
<p>Sufficient space in HDFS for at least two copies of the HBase data directory</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Before upgrade</div>
<p>Before beginning the upgrade process, you must take a complete backup of HBase&#8217;s backing data. The following instructions cover backing up the data within the current HDFS instance. Alternatively, you can use the distcp command to copy the data to another HDFS cluster.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Stop the HBase cluster</p>
</li>
<li>
<p>Copy the HBase data directory to a backup location using the <a href="https://hadoop.apache.org/docs/current/hadoop-distcp/DistCp.html">distcp command</a> as the HDFS super user (shown below on a security enabled cluster)</p>
<div class="listingblock">
<div class="title">Using distcp to backup the HBase data directory</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">[hpnewton@gateway_node.example.com ~]$ kinit -k -t hdfs.keytab hdfs@EXAMPLE.COM
[hpnewton@gateway_node.example.com ~]$ hadoop distcp /hbase /hbase-pre-upgrade-backup</code></pre>
</div>
</div>
</li>
<li>
<p>Distcp will launch a mapreduce job to handle copying the files in a distributed fashion. Check the output of the distcp command to ensure this job completed successfully.</p>
</li>
</ol>
</div>
<div class="olist arabic">
<div class="title">Performing a rollback</div>
<ol class="arabic">
<li>
<p>Stop HBase</p>
</li>
<li>
<p>Perform a downgrade for HDFS and a downgrade/rollback for ZooKeeper (HBase should remain stopped)</p>
</li>
<li>
<p>Change the installed version of HBase to the previous version</p>
</li>
<li>
<p>Restore the HBase data directory from prior to the upgrade as the HDFS super user (shown below on a security enabled cluster). If you backed up your data on another HDFS cluster instead of locally, you will need to use the distcp command to copy it back to the current HDFS cluster.</p>
<div class="listingblock">
<div class="title">Restore the HBase data directory</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">[hpnewton@gateway_node.example.com ~]$ kinit -k -t hdfs.keytab hdfs@EXAMPLE.COM
[hpnewton@gateway_node.example.com ~]$ hdfs dfs -mv /hbase /hbase-upgrade-rollback
[hpnewton@gateway_node.example.com ~]$ hdfs dfs -mv /hbase-pre-upgrade-backup /hbase</code></pre>
</div>
</div>
</li>
<li>
<p>Clean out ZooKeeper information related to HBase. WARNING: This step will permanently destroy all replication peers. Please see the section on HBase Replication under Caveats for more information.</p>
<div class="listingblock">
<div class="title">Clean HBase information out of ZooKeeper</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">[hpnewton@gateway_node.example.com ~]$ zookeeper-client -server zookeeper1.example.com:2181,zookeeper2.example.com:2181,zookeeper3.example.com:2181
Welcome to ZooKeeper!
JLine support is disabled
rmr /hbase
quit
Quitting...</code></pre>
</div>
</div>
</li>
<li>
<p>Start HBase</p>
</li>
<li>
<p>Verify HBase contents–use the HBase shell to list tables and scan some known values.</p>
</li>
</ol>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_upgrade_paths"><a class="anchor" href="#_upgrade_paths"></a>13. Upgrade Paths</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="upgrade2.2"><a class="anchor" href="#upgrade2.2"></a>13.1. Upgrade from 2.0 or 2.1 to 2.2+</h3>
<div class="paragraph">
<p>HBase 2.2+ uses a new Procedure form assiging/unassigning/moving Regions. It does not process HBase 2.1 and 2.0&#8217;s Unassign/Assign Procedure types. Upgrade requires that we first drain the Master Procedure Store of old style Procedures before starting the new 2.2 Master. So you need to make sure that before you kill the old version (2.0 or 2.1) Master, there is no region in transition. And once the new version (2.2+) Master is up, you can rolling upgrade RegionServers one by one.</p>
</div>
<div class="paragraph">
<p>And there is a more safer way if you are running 2.1.1+ or 2.0.3+ cluster. It need four steps to upgrade Master.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Shutdown both active and standby Masters (Your cluster will continue to server reads and writes without interruption).</p>
</li>
<li>
<p>Set the property hbase.procedure.upgrade-to-2-2 to true in hbase-site.xml for the Master, and start only one Master, still using the 2.1.1+ (or 2.0.3+) version.</p>
</li>
<li>
<p>Wait until the Master quits. Confirm that there is a 'READY TO ROLLING UPGRADE' message in the Master log as the cause of the shutdown. The Procedure Store is now empty.</p>
</li>
<li>
<p>Start new Masters with the new 2.2+ version.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Then you can rolling upgrade RegionServers one by one. See <a href="https://issues.apache.org/jira/browse/HBASE-21075">HBASE-21075</a> for more details.</p>
</div>
</div>
<div class="sect2">
<h3 id="upgrade2.0"><a class="anchor" href="#upgrade2.0"></a>13.2. Upgrading from 1.x to 2.x</h3>
<div class="paragraph">
<p>In this section we will first call out significant changes compared to the prior stable HBase release and then go over the upgrade process. Be sure to read the former with care so you avoid suprises.</p>
</div>
<div class="sect3">
<h4 id="_changes_of_note"><a class="anchor" href="#_changes_of_note"></a>13.2.1. Changes of Note!</h4>
<div class="paragraph">
<p>First we&#8217;ll cover deployment / operational changes that you might hit when upgrading to HBase 2.0+. After that we&#8217;ll call out changes for downstream applications. Please note that Coprocessors are covered in the operational section. Also note that this section is not meant to convey information about new features that may be of interest to you. For a complete summary of changes, please see the CHANGES.txt file in the source release artifact for the version you are planning to upgrade to.</p>
</div>
<div id="upgrade2.0.basic.requirements" class="paragraph">
<div class="title">Update to basic prerequisite minimums in HBase 2.0+</div>
<p>As noted in the section <a href="#basic.prerequisites">Basic Prerequisites</a>, HBase 2.0+ requires a minimum of Java 8 and Hadoop 2.6. The HBase community recommends ensuring you have already completed any needed upgrades in prerequisites prior to upgrading your HBase version.</p>
</div>
<div id="upgrade2.0.hbck" class="paragraph">
<div class="title">HBCK must match HBase server version</div>
<p>You <strong>must not</strong> use an HBase 1.x version of HBCK against an HBase 2.0+ cluster. HBCK is strongly tied to the HBase server version. Using the HBCK tool from an earlier release against an HBase 2.0+ cluster will destructively alter said cluster in unrecoverable ways.</p>
</div>
<div class="paragraph">
<p>As of HBase 2.0, HBCK (A.K.A <em>HBCK1</em> or <em>hbck1</em>) is a read-only tool that can report the status of some non-public system internals but will often misread state because it does not understand the workings of hbase2.</p>
</div>
<div class="paragraph">
<p>To read about HBCK&#8217;s replacement, see <a href="#HBCK2">HBase <code>HBCK2</code></a> in <a href="#ops_mgt">Apache HBase Operational Management</a>.</p>
</div>
<div class="admonitionblock important">
<table>
<tr>
<td class="icon">
<i class="fa icon-important" title="Important"></i>
</td>
<td class="content">
Related, before you upgrade, ensure that <em>hbck1</em> reports no <code>INCONSISTENCIES</code>. Fixing hbase1-type inconsistencies post-upgrade is an involved process.
</td>
</tr>
</table>
</div>
<div id="upgrade2.0.removed.configs" class="paragraph">
<div class="title">Configuration settings no longer in HBase 2.0+</div>
<p>The following configuration settings are no longer applicable or available. For details, please see the detailed release notes.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>hbase.config.read.zookeeper.config (see <a href="#upgrade2.0.zkconfig">ZooKeeper configs no longer read from zoo.cfg</a> for migration details)</p>
</li>
<li>
<p>hbase.zookeeper.useMulti (HBase now always uses ZK&#8217;s multi functionality)</p>
</li>
<li>
<p>hbase.rpc.client.threads.max</p>
</li>
<li>
<p>hbase.rpc.client.nativetransport</p>
</li>
<li>
<p>hbase.fs.tmp.dir</p>
</li>
<li>
<p>hbase.bucketcache.combinedcache.enabled</p>
</li>
<li>
<p>hbase.bucketcache.ioengine no longer supports the 'heap' value.</p>
</li>
<li>
<p>hbase.bulkload.staging.dir</p>
</li>
<li>
<p>hbase.balancer.tablesOnMaster wasn&#8217;t removed, strictly speaking, but its meaning has fundamentally changed and users should not set it. See the section <a href="#upgrade2.0.regions.on.master">"Master hosting regions" feature broken and unsupported</a> for details.</p>
</li>
<li>
<p>hbase.master.distributed.log.replay See the section <a href="#upgrade2.0.distributed.log.replay">"Distributed Log Replay" feature broken and removed</a> for details</p>
</li>
<li>
<p>hbase.regionserver.disallow.writes.when.recovering See the section <a href="#upgrade2.0.distributed.log.replay">"Distributed Log Replay" feature broken and removed</a> for details</p>
</li>
<li>
<p>hbase.regionserver.wal.logreplay.batch.size See the section <a href="#upgrade2.0.distributed.log.replay">"Distributed Log Replay" feature broken and removed</a> for details</p>
</li>
<li>
<p>hbase.master.catalog.timeout</p>
</li>
<li>
<p>hbase.regionserver.catalog.timeout</p>
</li>
<li>
<p>hbase.metrics.exposeOperationTimes</p>
</li>
<li>
<p>hbase.metrics.showTableName</p>
</li>
<li>
<p>hbase.online.schema.update.enable (HBase now always supports this)</p>
</li>
<li>
<p>hbase.thrift.htablepool.size.max</p>
</li>
</ul>
</div>
<div id="upgrade2.0.renamed.configs" class="paragraph">
<div class="title">Configuration properties that were renamed in HBase 2.0+</div>
<p>The following properties have been renamed. Attempts to set the old property will be ignored at run time.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 5. Renamed properties</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Old name</th>
<th class="tableblock halign-left valign-top">New name</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.rpc.server.nativetransport</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.netty.nativetransport</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.netty.rpc.server.worker.count</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.netty.worker.count</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hfile.compactions.discharger.interval</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hfile.compaction.discharger.interval</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hregion.percolumnfamilyflush.size.lower.bound</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase.hregion.percolumnfamilyflush.size.lower.bound.min</p></td>
</tr>
</tbody>
</table>
<div id="upgrade2.0.changed.defaults" class="paragraph">
<div class="title">Configuration settings with different defaults in HBase 2.0+</div>
<p>The following configuration settings changed their default value. Where applicable, the value to set to restore the behavior of HBase 1.2 is given.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>hbase.security.authorization now defaults to false. set to true to restore same behavior as previous default.</p>
</li>
<li>
<p>hbase.client.retries.number is now set to 10. Previously it was 35. Downstream users are advised to use client timeouts as described in section <a href="#config_timeouts">Timeout settings</a> instead.</p>
</li>
<li>
<p>hbase.client.serverside.retries.multiplier is now set to 3. Previously it was 10. Downstream users are advised to use client timesout as describe in section <a href="#config_timeouts">Timeout settings</a> instead.</p>
</li>
<li>
<p>hbase.master.fileSplitTimeout is now set to 10 minutes. Previously it was 30 seconds.</p>
</li>
<li>
<p>hbase.regionserver.logroll.multiplier is now set to 0.5. Previously it was 0.95. This change is tied with the following doubling of block size. Combined, these two configuration changes should make for WALs of about the same size as those in hbase-1.x but there should be less incidence of small blocks because we fail to roll the WAL before we hit the blocksize threshold. See <a href="https://issues.apache.org/jira/browse/HBASE-19148">HBASE-19148</a> for discussion.</p>
</li>
<li>
<p>hbase.regionserver.hlog.blocksize defaults to 2x the HDFS default block size for the WAL dir. Previously it was equal to the HDFS default block size for the WAL dir.</p>
</li>
<li>
<p>hbase.client.start.log.errors.counter changed to 5. Previously it was 9.</p>
</li>
<li>
<p>hbase.ipc.server.callqueue.type changed to 'fifo'. In HBase versions 1.0 - 1.2 it was 'deadline'. In prior and later 1.x versions it already defaults to 'fifo'.</p>
</li>
<li>
<p>hbase.hregion.memstore.chunkpool.maxsize is 1.0 by default. Previously it was 0.0. Effectively, this means previously we would not use a chunk pool when our memstore is onheap and now we will. See the section <a href="#gcpause">Long GC pauses</a> for more infromation about the MSLAB chunk pool.</p>
</li>
<li>
<p>hbase.master.cleaner.interval is now set to 10 minutes. Previously it was 1 minute.</p>
</li>
<li>
<p>hbase.master.procedure.threads will now default to 1/4 of the number of available CPUs, but not less than 16 threads. Previously it would be number of threads equal to number of CPUs.</p>
</li>
<li>
<p>hbase.hstore.blockingStoreFiles is now 16. Previously it was 10.</p>
</li>
<li>
<p>hbase.http.max.threads is now 16. Previously it was 10.</p>
</li>
<li>
<p>hbase.client.max.perserver.tasks is now 2. Previously it was 5.</p>
</li>
<li>
<p>hbase.normalizer.period is now 5 minutes. Previously it was 30 minutes.</p>
</li>
<li>
<p>hbase.regionserver.region.split.policy is now SteppingSplitPolicy. Previously it was IncreasingToUpperBoundRegionSplitPolicy.</p>
</li>
<li>
<p>replication.source.ratio is now 0.5. Previously it was 0.1.</p>
</li>
</ul>
</div>
<div id="upgrade2.0.regions.on.master" class="paragraph">
<div class="title">"Master hosting regions" feature broken and unsupported</div>
<p>The feature "Master acts as region server" and associated follow-on work available in HBase 1.y is non-functional in HBase 2.y and should not be used in a production setting due to deadlock on Master initialization. Downstream users are advised to treat related configuration settings as experimental and the feature as inappropriate for production settings.</p>
</div>
<div class="paragraph">
<p>A brief summary of related changes:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Master no longer carries regions by default</p>
</li>
<li>
<p>hbase.balancer.tablesOnMaster is a boolean, default false (if it holds an HBase 1.x list of tables, will default to false)</p>
</li>
<li>
<p>hbase.balancer.tablesOnMaster.systemTablesOnly is boolean to keep user tables off master. default false</p>
</li>
<li>
<p>those wishing to replicate old list-of-servers config should deploy a stand-alone RegionServer process and then rely on Region Server Groups</p>
</li>
</ul>
</div>
<div id="upgrade2.0.distributed.log.replay" class="paragraph">
<div class="title">"Distributed Log Replay" feature broken and removed</div>
<p>The Distributed Log Replay feature was broken and has been removed from HBase 2.y+. As a consequence all related configs, metrics, RPC fields, and logging have also been removed. Note that this feature was found to be unreliable in the run up to HBase 1.0, defaulted to being unused, and was effectively removed in HBase 1.2.0 when we started ignoring the config that turns it on (<a href="https://issues.apache.org/jira/browse/HBASE-14465">HBASE-14465</a>). If you are currently using the feature, be sure to perform a clean shutdown, ensure all DLR work is complete, and disable the feature prior to upgrading.</p>
</div>
<div id="upgrade2.0.prefix-tree.removed" class="paragraph">
<div class="title"><em>prefix-tree</em> encoding removed</div>
<p>The prefix-tree encoding was removed from HBase 2.0.0 (<a href="https://issues.apache.org/jira/browse/HBASE-19179">HBASE-19179</a>).
It was (late!) deprecated in hbase-1.2.7, hbase-1.4.0, and hbase-1.3.2.</p>
</div>
<div class="paragraph">
<p>This feature was removed because it as not being actively maintained. If interested in reviving this
sweet facility which improved random read latencies at the expensive of slowed writes,
write the HBase developers list at <em>dev at hbase dot apache dot org</em>.</p>
</div>
<div class="paragraph">
<p>The prefix-tree encoding needs to be removed from all tables before upgrading to HBase 2.0+.
To do that first you need to change the encoding from PREFIX_TREE to something else that is supported in HBase 2.0.
After that you have to major compact the tables that were using PREFIX_TREE encoding before.
To check which column families are using incompatible data block encoding you can use <a href="#ops.pre-upgrade">Pre-Upgrade Validator</a>.</p>
</div>
<div id="upgrade2.0.metrics" class="paragraph">
<div class="title">Changed metrics</div>
<p>The following metrics have changed names:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Metrics previously published under the name "AssignmentManger" [sic] are now published under the name "AssignmentManager"</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following metrics have changed their meaning:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The metric 'blockCacheEvictionCount' published on a per-region server basis no longer includes blocks removed from the cache due to the invalidation of the hfiles they are from (e.g. via compaction).</p>
</li>
<li>
<p>The metric 'totalRequestCount' increments once per request; previously it incremented by the number of <code>Actions</code> carried in the request; e.g. if a request was a <code>multi</code> made of four Gets and two Puts, we&#8217;d increment 'totalRequestCount' by six; now we increment by one regardless. Expect to see lower values for this metric in hbase-2.0.0.</p>
</li>
<li>
<p>The 'readRequestCount' now counts reads that return a non-empty row where in older hbases, we&#8217;d increment 'readRequestCount' whether a Result or not. This change will flatten the profile of the read-requests graphs if requests for non-existent rows. A YCSB read-heavy workload can do this dependent on how the database was loaded.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following metrics have been removed:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Metrics related to the Distributed Log Replay feature are no longer present. They were previsouly found in the region server context under the name 'replay'. See the section <a href="#upgrade2.0.distributed.log.replay">"Distributed Log Replay" feature broken and removed</a> for details.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following metrics have been added:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>'totalRowActionRequestCount' is a count of region row actions summing reads and writes.</p>
</li>
</ul>
</div>
<div id="upgrade2.0.logging" class="paragraph">
<div class="title">Changed logging</div>
<p>HBase-2.0.0 now uses <a href="https://www.slf4j.org/">slf4j</a> as its logging frontend.
Prevously, we used <a href="http://logging.apache.org/log4j/1.2/">log4j (1.2)</a>.
For most the transition should be seamless; slf4j does a good job interpreting
<em>log4j.properties</em> logging configuration files such that you should not notice
any difference in your log system emissions.</p>
</div>
<div class="paragraph">
<p>That said, your <em>log4j.properties</em> may need freshening. See <a href="https://issues.apache.org/jira/browse/HBASE-20351">HBASE-20351</a>
for example, where a stale log configuration file manifest as netty configuration
being dumped at DEBUG level as preamble on every shell command invocation.</p>
</div>
<div id="upgrade2.0.zkconfig" class="paragraph">
<div class="title">ZooKeeper configs no longer read from zoo.cfg</div>
<p>HBase no longer optionally reads the 'zoo.cfg' file for ZooKeeper related configuration settings. If you previously relied on the 'hbase.config.read.zookeeper.config' config for this functionality, you should migrate any needed settings to the hbase-site.xml file while adding the prefix 'hbase.zookeeper.property.' to each property name.</p>
</div>
<div id="upgrade2.0.permissions" class="paragraph">
<div class="title">Changes in permissions</div>
<p>The following permission related changes either altered semantics or defaults:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Permissions granted to a user now merge with existing permissions for that user, rather than over-writing them. (see <a href="https://issues.apache.org/jira/browse/HBASE-17472">the release note on HBASE-17472</a> for details)</p>
</li>
<li>
<p>Region Server Group commands (added in 1.4.0) now require admin privileges.</p>
</li>
</ul>
</div>
<div id="upgrade2.0.admin.commands" class="paragraph">
<div class="title">Most Admin APIs don&#8217;t work against an HBase 2.0+ cluster from pre-HBase 2.0 clients</div>
<p>A number of admin commands are known to not work when used from a pre-HBase 2.0 client. This includes an HBase Shell that has the library jars from pre-HBase 2.0. You will need to plan for an outage of use of admin APIs and commands until you can also update to the needed client version.</p>
</div>
<div class="paragraph">
<p>The following client operations do not work against HBase 2.0+ cluster when executed from a pre-HBase 2.0 client:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>list_procedures</p>
</li>
<li>
<p>split</p>
</li>
<li>
<p>merge_region</p>
</li>
<li>
<p>list_quotas</p>
</li>
<li>
<p>enable_table_replication</p>
</li>
<li>
<p>disable_table_replication</p>
</li>
<li>
<p>Snapshot related commands</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Deprecated in 1.0 admin commands have been removed.</div>
<p>The following commands that were deprecated in 1.0 have been removed. Where applicable the replacement command is listed.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The 'hlog' command has been removed. Downstream users should rely on the 'wal' command instead.</p>
</li>
</ul>
</div>
<div id="upgrade2.0.memory" class="paragraph">
<div class="title">Region Server memory consumption changes.</div>
<p>Users upgrading from versions prior to HBase 1.4 should read the instructions in section <a href="#upgrade1.4.memory">Region Server memory consumption changes.</a>.</p>
</div>
<div class="paragraph">
<p>Additionally, HBase 2.0 has changed how memstore memory is tracked for flushing decisions. Previously, both the data size and overhead for storage were used to calculate utilization against the flush threashold. Now, only data size is used to make these per-region decisions. Globally the addition of the storage overhead is used to make decisions about forced flushes.</p>
</div>
<div id="upgrade2.0.ui.splitmerge.by.row" class="paragraph">
<div class="title">Web UI for splitting and merging operate on row prefixes</div>
<p>Previously, the Web UI included functionality on table status pages to merge or split based on an encoded region name. In HBase 2.0, instead this functionality works by taking a row prefix.</p>
</div>
<div id="upgrade2.0.replication" class="paragraph">
<div class="title">Special upgrading for Replication users from pre-HBase 1.4</div>
<p>User running versions of HBase prior to the 1.4.0 release that make use of replication should be sure to read the instructions in the section <a href="#upgrade1.4.replication">Replication peer&#8217;s TableCFs config</a>.</p>
</div>
<div id="upgrade2.0.shell" class="paragraph">
<div class="title">HBase shell changes</div>
<p>The HBase shell command relies on a bundled JRuby instance. This bundled JRuby been updated from version 1.6.8 to version 9.1.10.0. The represents a change from Ruby 1.8 to Ruby 2.3.3, which introduces non-compatible language changes for user scripts.</p>
</div>
<div class="paragraph">
<p>The HBase shell command now ignores the '--return-values' flag that was present in early HBase 1.4 releases. Instead the shell always behaves as though that flag were passed. If you wish to avoid having expression results printed in the console you should alter your IRB configuration as noted in the section <a href="#irbrc"><em>irbrc</em></a>.</p>
</div>
<div id="upgrade2.0.coprocessors" class="paragraph">
<div class="title">Coprocessor APIs have changed in HBase 2.0+</div>
<p>All Coprocessor APIs have been refactored to improve supportability around binary API compatibility for future versions of HBase. If you or applications you rely on have custom HBase coprocessors, you should read <a href="https://issues.apache.org/jira/browse/HBASE-18169">the release notes for HBASE-18169</a> for details of changes you will need to make prior to upgrading to HBase 2.0+.</p>
</div>
<div class="paragraph">
<p>For example, if you had a BaseRegionObserver in HBase 1.2 then at a minimum you will need to update it to implement both RegionObserver and RegionCoprocessor and add the method</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
<span class="annotation">@Override</span>
<span class="directive">public</span> Optional&lt;RegionObserver&gt; getRegionObserver() {
<span class="keyword">return</span> Optional.of(<span class="local-variable">this</span>);
}
...</code></pre>
</div>
</div>
<div id="upgrade2.0.hfile3.only" class="paragraph">
<div class="title">HBase 2.0+ can no longer write HFile v2 files.</div>
<p>HBase has simplified our internal HFile handling. As a result, we can no longer write HFile versions earlier than the default of version 3. Upgrading users should ensure that hfile.format.version is not set to 2 in hbase-site.xml before upgrading. Failing to do so will cause Region Server failure. HBase can still read HFiles written in the older version 2 format.</p>
</div>
<div id="upgrade2.0.pb.wal.only" class="paragraph">
<div class="title">HBase 2.0+ can no longer read Sequence File based WAL file.</div>
<p>HBase can no longer read the deprecated WAL files written in the Apache Hadoop Sequence File format. The hbase.regionserver.hlog.reader.impl and hbase.regionserver.hlog.reader.impl configuration entries should be set to use the Protobuf based WAL reader / writer classes. This implementation has been the default since HBase 0.96, so legacy WAL files should not be a concern for most downstream users.</p>
</div>
<div class="paragraph">
<p>A clean cluster shutdown should ensure there are no WAL files. If you are unsure of a given WAL file&#8217;s format you can use the <code>hbase wal</code> command to parse files while the HBase cluster is offline. In HBase 2.0+, this command will not be able to read a Sequence File based WAL. For more information on the tool see the section <a href="#hlog_tool.prettyprint">WALPrettyPrinter</a>.</p>
</div>
<div id="upgrade2.0.filters" class="paragraph">
<div class="title">Change in behavior for filters</div>
<p>The Filter ReturnCode NEXT_ROW has been redefined as skipping to next row in current family, not to next row in all family. it’s more reasonable, because ReturnCode is a concept in store level, not in region level.</p>
</div>
<div id="upgrade2.0.shaded.client.preferred" class="paragraph">
<div class="title">Downstream HBase 2.0+ users should use the shaded client</div>
<p>Downstream users are strongly urged to rely on the Maven coordinates org.apache.hbase:hbase-shaded-client for their runtime use. This artifact contains all the needed implementation details for talking to an HBase cluster while minimizing the number of third party dependencies exposed.</p>
</div>
<div class="paragraph">
<p>Note that this artifact exposes some classes in the org.apache.hadoop package space (e.g. o.a.h.configuration.Configuration) so that we can maintain source compatibility with our public API. Those classes are included so that they can be altered to use the same relocated third party dependencies as the rest of the HBase client code. In the event that you need to <strong>also</strong> use Hadoop in your code, you should ensure all Hadoop related jars precede the HBase client jar in your classpath.</p>
</div>
<div id="upgrade2.0.mapreduce.module" class="paragraph">
<div class="title">Downstream HBase 2.0+ users of MapReduce must switch to new artifact</div>
<p>Downstream users of HBase&#8217;s integration for Apache Hadoop MapReduce must switch to relying on the org.apache.hbase:hbase-shaded-mapreduce module for their runtime use. Historically, downstream users relied on either the org.apache.hbase:hbase-server or org.apache.hbase:hbase-shaded-server artifacts for these classes. Both uses are no longer supported and in the vast majority of cases will fail at runtime.</p>
</div>
<div class="paragraph">
<p>Note that this artifact exposes some classes in the org.apache.hadoop package space (e.g. o.a.h.configuration.Configuration) so that we can maintain source compatibility with our public API. Those classes are included so that they can be altered to use the same relocated third party dependencies as the rest of the HBase client code. In the event that you need to <strong>also</strong> use Hadoop in your code, you should ensure all Hadoop related jars precede the HBase client jar in your classpath.</p>
</div>
<div id="upgrade2.0.dependencies" class="paragraph">
<div class="title">Significant changes to runtime classpath</div>
<p>A number of internal dependencies for HBase were updated or removed from the runtime classpath. Downstream client users who do not follow the guidance in <a href="#upgrade2.0.shaded.client.preferred">Downstream HBase 2.0+ users should use the shaded client</a> will have to examine the set of dependencies Maven pulls in for impact. Downstream users of LimitedPrivate Coprocessor APIs will need to examine the runtime environment for impact. For details on our new handling of third party libraries that have historically been a problem with respect to harmonizing compatible runtime versions, see the reference guide section <a href="#thirdparty">The hbase-thirdparty dependency and shading/relocation</a>.</p>
</div>
<div id="upgrade2.0.public.api" class="paragraph">
<div class="title">Multiple breaking changes to source and binary compatibility for client API</div>
<p>The Java client API for HBase has a number of changes that break both source and binary compatibility for details see the Compatibility Check Report for the release you&#8217;ll be upgrading to.</p>
</div>
<div id="upgrade2.0.tracing" class="paragraph">
<div class="title">Tracing implementation changes</div>
<p>The backing implementation of HBase&#8217;s tracing features was updated from Apache HTrace 3 to HTrace 4, which includes several breaking changes. While HTrace 3 and 4 can coexist in the same runtime, they will not integrate with each other, leading to disjoint trace information.</p>
</div>
<div class="paragraph">
<p>The internal changes to HBase during this upgrade were sufficient for compilation, but it has not been confirmed that there are no regressions in tracing functionality. Please consider this feature expiremental for the immediate future.</p>
</div>
<div class="paragraph">
<p>If you previously relied on client side tracing integrated with HBase operations, it is recommended that you upgrade your usage to HTrace 4 as well.</p>
</div>
<div id="upgrade2.0.hfile.compatability" class="paragraph">
<div class="title">HFile lose forward compatability</div>
<p>HFiles generated by 2.0.0, 2.0.1, 2.1.0 are not forward compatible to 1.4.6-, 1.3.2.1-, 1.2.6.1-,
and other inactive releases. Why HFile lose compatability is hbase in new versions
(2.0.0, 2.0.1, 2.1.0) use protobuf to serialize/deserialize TimeRangeTracker (TRT) while old
versions use DataInput/DataOutput. To solve this, We have to put
<a href="https://jira.apache.org/jira/browse/HBASE-21012">HBASE-21012</a>
to 2.x and put <a href="https://jira.apache.org/jira/browse/HBASE-21013">HBASE-21013</a> in 1.x.
For more information, please check
<a href="https://jira.apache.org/jira/browse/HBASE-21008">HBASE-21008</a>.</p>
</div>
<div id="upgrade2.0.perf" class="paragraph">
<div class="title">Performance</div>
<p>You will likely see a change in the performance profile on upgrade to hbase-2.0.0 given
read and write paths have undergone significant change. On release, writes may be
slower with reads about the same or much better, dependent on context. Be prepared
to spend time re-tuning (See <a href="#performance">Apache HBase Performance Tuning</a>).
Performance is also an area that is now under active review so look forward to
improvement in coming releases (See
<a href="https://issues.apache.org/jira/browse/HBASE-20188">HBASE-20188 TESTING Performance</a>).</p>
</div>
<div id="upgrade2.0.it.kerberos" class="paragraph">
<div class="title">Integration Tests and Kerberos</div>
<p>Integration Tests (<code>IntegrationTests*</code>) used to rely on the Kerberos credential cache
for authentication against secured clusters. This used to lead to tests failing due
to authentication failures when the tickets in the credential cache expired.
As of hbase-2.0.0 (and hbase-1.3.0+), the integration test clients will make use
of the configuration properties <code>hbase.client.keytab.file</code> and
<code>hbase.client.kerberos.principal</code>. They are required. The clients will perform a
login from the configured keytab file and automatically refresh the credentials
in the background for the process lifetime (See
<a href="https://issues.apache.org/jira/browse/HBASE-16231">HBASE-16231</a>).</p>
</div>
<div id="upgrade2.0.compaction.throughput.limit" class="paragraph">
<div class="title">Default Compaction Throughput</div>
<p>HBase 2.x comes with default limits to the speed at which compactions can execute. This
limit is defined per RegionServer. In previous versions of HBase earlier than 1.5, there
was no limit to the speed at which a compaction could run by default. Applying a limit
to the throughput of a compaction should ensure more stable operations from RegionServers.</p>
</div>
<div class="paragraph">
<p>Take care to notice that this limit is <em>per RegionServer</em>, not <em>per compaction</em>.</p>
</div>
<div class="paragraph">
<p>The throughput limit is defined as a range of bytes written per second, and is
allowed to vary within the given lower and upper bound. RegionServers observe the
current throughput of a compaction and apply a linear formula to adjust the allowed
throughput, within the lower and upper bound, with respect to external pressure.
For compactions, external pressure is defined as the number of store files with
respect to the maximum number of allowed store files. The more store files, the
higher the compaction pressure.</p>
</div>
<div class="paragraph">
<p>Configuration of this throughput is governed by the following properties.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The lower bound is defined by <code>hbase.hstore.compaction.throughput.lower.bound</code>
and defaults to 50 MB/s (<code>52428800</code>).</p>
</li>
<li>
<p>The upper bound is defined by <code>hbase.hstore.compaction.throughput.higher.bound</code>
and defaults to 100 MB/s (<code>104857600</code>).</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>To revert this behavior to the unlimited compaction throughput of earlier versions
of HBase, please set the following property to the implementation that applies no
limits to compactions.</p>
</div>
<div class="paragraph">
<p><code>hbase.regionserver.throughput.controller=org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController</code></p>
</div>
</div>
<div class="sect3">
<h4 id="upgrade2.0.coprocessors"><a class="anchor" href="#upgrade2.0.coprocessors"></a>13.2.2. Upgrading Coprocessors to 2.0</h4>
<div class="paragraph">
<p>Coprocessors have changed substantially in 2.0 ranging from top level design changes in class
hierarchies to changed/removed methods, interfaces, etc.
(Parent jira: <a href="https://issues.apache.org/jira/browse/HBASE-18169">HBASE-18169 Coprocessor fix
and cleanup before 2.0.0 release</a>). Some of the reasons for such widespread changes:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Pass Interfaces instead of Implementations; e.g. TableDescriptor instead of HTableDescriptor and
Region instead of HRegion (<a href="https://issues.apache.org/jira/browse/HBASE-18241">HBASE-18241</a>
Change client.Table and client.Admin to not use HTableDescriptor).</p>
</li>
<li>
<p>Design refactor so implementers need to fill out less boilerplate and so we can do more
compile-time checking (<a href="https://issues.apache.org/jira/browse/HBASE-17732">HBASE-17732</a>)</p>
</li>
<li>
<p>Purge Protocol Buffers from Coprocessor API
(<a href="https://issues.apache.org/jira/browse/HBASE-18859">HBASE-18859</a>,
<a href="https://issues.apache.org/jira/browse/HBASE-16769">HBASE-16769</a>, etc)</p>
</li>
<li>
<p>Cut back on what we expose to Coprocessors removing hooks on internals that were too private to
expose (for eg. <a href="https://issues.apache.org/jira/browse/HBASE-18453">HBASE-18453</a>
CompactionRequest should not be exposed to user directly;
<a href="https://issues.apache.org/jira/browse/HBASE-18298">HBASE-18298</a> RegionServerServices Interface
cleanup for CP expose; etc)</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>To use coprocessors in 2.0, they should be rebuilt against new API otherwise they will fail to
load and HBase processes will die.</p>
</div>
<div class="paragraph">
<p>Suggested order of changes to upgrade the coprocessors:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Directly implement observer interfaces instead of extending Base*Observer classes. Change
<code>Foo extends BaseXXXObserver</code> to <code>Foo implements XXXObserver</code>.
(<a href="https://issues.apache.org/jira/browse/HBASE-17312">HBASE-17312</a>).</p>
</li>
<li>
<p>Adapt to design change from Inheritence to Composition
(<a href="https://issues.apache.org/jira/browse/HBASE-17732">HBASE-17732</a>) by following
<a href="https://github.com/apache/hbase/blob/master/dev-support/design-docs/Coprocessor_Design_Improvements-Use_composition_instead_of_inheritance-HBASE-17732.adoc#migrating-existing-cps-to-new-design">this
example</a>.</p>
</li>
<li>
<p>getTable() has been removed from the CoprocessorEnvrionment, coprocessors should self-manage
Table instances.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Some examples of writing coprocessors with new API can be found in hbase-example module
<a href="https://github.com/apache/hbase/tree/branch-2.0/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example">here</a> .</p>
</div>
<div class="paragraph">
<p>Lastly, if an api has been changed/removed that breaks you in an irreparable way, and if there&#8217;s a
good justification to add it back, bring it our notice (<a href="mailto:dev@hbase.apache.org">dev@hbase.apache.org</a>).</p>
</div>
</div>
<div class="sect3">
<h4 id="upgrade2.0.rolling.upgrades"><a class="anchor" href="#upgrade2.0.rolling.upgrades"></a>13.2.3. Rolling Upgrade from 1.x to 2.x</h4>
<div class="paragraph">
<p>Rolling upgrades are currently an experimental feature.
They have had limited testing. There are likely corner
cases as yet uncovered in our
limited experience so you should be careful if you go this
route. The stop/upgrade/start described in the next section,
<a href="#upgrade2.0.process">Upgrade process from 1.x to 2.x</a>, is the safest route.</p>
</div>
<div class="paragraph">
<p>That said, the below is a prescription for a
rolling upgrade of a 1.4 cluster.</p>
</div>
<div class="ulist">
<div class="title">Pre-Requirements</div>
<ul>
<li>
<p>Upgrade to the latest 1.4.x release. Pre 1.4 releases may also work but are not tested, so please upgrade to 1.4.3+ before upgrading to 2.x, unless you are an expert and familiar with the region assignment and crash processing. See the section <a href="#upgrade1.4">Upgrading from pre-1.4 to 1.4+</a> on how to upgrade to 1.4.x.</p>
</li>
<li>
<p>Make sure that the zk-less assignment is enabled, i.e, set <code>hbase.assignment.usezk</code> to <code>false</code>. This is the most important thing. It allows the 1.x master to assign/unassign regions to/from 2.x region servers. See the release note section of <a href="https://issues.apache.org/jira/browse/HBASE-11059">HBASE-11059</a> on how to migrate from zk based assignment to zk less assignment.</p>
</li>
<li>
<p>Before you upgrade, ensure that <em>hbck1</em> reports no <code>INCONSISTENCIES</code>. Fixing hbase1-type inconsistencies post-upgrade is an involved process.</p>
</li>
<li>
<p>We have tested rolling upgrading from 1.4.3 to 2.1.0, but it should also work if you want to upgrade to 2.0.x.</p>
</li>
</ul>
</div>
<div class="olist arabic">
<div class="title">Instructions</div>
<ol class="arabic">
<li>
<p>Unload a region server and upgrade it to 2.1.0. With <a href="https://issues.apache.org/jira/browse/HBASE-17931">HBASE-17931</a> in place, the meta region and regions for other system tables will be moved to this region server immediately. If not, please move them manually to the new region server. This is very important because</p>
<div class="ulist">
<ul>
<li>
<p>The schema of meta region is hard coded, if meta is on an old region server, then the new region servers can not access it as it does not have some families, for example, table state.</p>
</li>
<li>
<p>Client with lower version can communicate with server with higher version, but not vice versa. If the meta region is on an old region server, the new region server will use a client with higher version to communicate with a server with lower version, this may introduce strange problems.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Rolling upgrade all other region servers.</p>
</li>
<li>
<p>Upgrading masters.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>It is OK that during the rolling upgrading there are region server crashes. The 1.x master can assign regions to both 1.x and 2.x region servers, and <a href="https://issues.apache.org/jira/browse/HBASE-19166">HBASE-19166</a> fixed a problem so that 1.x region server can also read the WALs written by 2.x region server and split them.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
please read the <a href="#_changes_of_note">Changes of Note!</a> section carefully before rolling upgrading. Make sure that you do not use the removed features in 2.0, for example, the prefix-tree encoding, the old hfile format, etc. They could both fail the upgrading and leave the cluster in an intermediate state and hard to recover.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If you have success running this prescription, please notify the dev list with a note on your experience and/or update the above with any deviations you may have taken so others going this route can benefit from your efforts.
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="upgrade2.0.process"><a class="anchor" href="#upgrade2.0.process"></a>13.2.4. Upgrade process from 1.x to 2.x</h4>
<div class="paragraph">
<p>To upgrade an existing HBase 1.x cluster, you should:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Ensure that <em>hbck1</em> reports no <code>INCONSISTENCIES</code>. Fixing hbase1-type inconsistencies post-upgrade is an involved process. Fix all <em>hbck1</em> complaints before proceeding.</p>
</li>
<li>
<p>Clean shutdown of existing 1.x cluster</p>
</li>
<li>
<p>Update coprocessors</p>
</li>
<li>
<p>Upgrade Master roles first</p>
</li>
<li>
<p>Upgrade RegionServers</p>
</li>
<li>
<p>(Eventually) Upgrade Clients</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect2">
<h3 id="upgrade1.4"><a class="anchor" href="#upgrade1.4"></a>13.3. Upgrading from pre-1.4 to 1.4+</h3>
<div class="sect3">
<h4 id="upgrade1.4.memory"><a class="anchor" href="#upgrade1.4.memory"></a>13.3.1. Region Server memory consumption changes.</h4>
<div class="paragraph">
<p>Users upgrading from versions prior to HBase 1.4 should be aware that the estimates of heap usage by the memstore objects (KeyValue, object and array header sizes, etc) have been made more accurate for heap sizes up to 32G (using CompressedOops), resulting in them dropping by 10-50% in practice. This also results in less number of flushes and compactions due to "fatter" flushes. YMMV. As a result, the actual heap usage of the memstore before being flushed may increase by up to 100%. If configured memory limits for the region server had been tuned based on observed usage, this change could result in worse GC behavior or even OutOfMemory errors. Set the environment property (not hbase-site.xml) "hbase.memorylayout.use.unsafe" to false to disable.</p>
</div>
</div>
<div class="sect3">
<h4 id="upgrade1.4.replication"><a class="anchor" href="#upgrade1.4.replication"></a>13.3.2. Replication peer&#8217;s TableCFs config</h4>
<div class="paragraph">
<p>Before 1.4, the table name can&#8217;t include namespace for replication peer&#8217;s TableCFs config. It was fixed by add TableCFs to ReplicationPeerConfig which was stored on Zookeeper. So when upgrade to 1.4, you have to update the original ReplicationPeerConfig data on Zookeeper firstly. There are four steps to upgrade when your cluster have a replication peer with TableCFs config.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Disable the replication peer.</p>
</li>
<li>
<p>If master has permission to write replication peer znode, then rolling update master directly. If not, use TableCFsUpdater tool to update the replication peer&#8217;s config.</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.replication.master.TableCFsUpdater update</pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Rolling update regionservers.</p>
</li>
<li>
<p>Enable the replication peer.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Notes:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Can&#8217;t use the old client(before 1.4) to change the replication peer&#8217;s config. Because the client will write config to Zookeeper directly, the old client will miss TableCFs config. And the old client write TableCFs config to the old tablecfs znode, it will not work for new version regionserver.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="upgrade1.4.rawscan"><a class="anchor" href="#upgrade1.4.rawscan"></a>13.3.3. Raw scan now ignores TTL</h4>
<div class="paragraph">
<p>Doing a raw scan will now return results that have expired according to TTL settings.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="upgrade1.3"><a class="anchor" href="#upgrade1.3"></a>13.4. Upgrading from pre-1.3 to 1.3+</h3>
<div class="paragraph">
<p>If running Integration Tests under Kerberos, see <a href="#upgrade2.0.it.kerberos">Integration Tests and Kerberos</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="upgrade1.0"><a class="anchor" href="#upgrade1.0"></a>13.5. Upgrading to 1.x</h3>
<div class="paragraph">
<p>Please consult the documentation published specifically for the version of HBase that you are upgrading to for details on the upgrade process.</p>
</div>
</div>
</div>
</div>
<h1 id="shell" class="sect0"><a class="anchor" href="#shell"></a>The Apache HBase Shell</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>The Apache HBase Shell is <a href="http://jruby.org">(J)Ruby</a>'s IRB with some HBase particular commands added.
Anything you can do in IRB, you should be able to do in the HBase Shell.</p>
</div>
<div class="paragraph">
<p>To run the HBase shell, do as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./bin/hbase shell</code></pre>
</div>
</div>
<div class="paragraph">
<p>Type <code>help</code> and then <code>&lt;RETURN&gt;</code> to see a listing of shell commands and options.
Browse at least the paragraphs at the end of the help output for the gist of how variables and command arguments are entered into the HBase shell; in particular note how table names, rows, and columns, etc., must be quoted.</p>
</div>
<div class="paragraph">
<p>See <a href="#shell_exercises">shell exercises</a> for example basic shell operation.</p>
</div>
<div class="paragraph">
<p>Here is a nicely formatted listing of <a href="http://learnhbase.wordpress.com/2013/03/02/hbase-shell-commands/">all shell
commands</a> by Rajeshbabu Chintaguntla.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="scripting"><a class="anchor" href="#scripting"></a>14. Scripting with Ruby</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For examples scripting Apache HBase, look in the HBase <em>bin</em> directory.
Look at the files that end in <em>*.rb</em>.
To run one of these files, do as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./bin/hbase org.jruby.Main PATH_TO_SCRIPT</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_running_the_shell_in_non_interactive_mode"><a class="anchor" href="#_running_the_shell_in_non_interactive_mode"></a>15. Running the Shell in Non-Interactive Mode</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A new non-interactive mode has been added to the HBase Shell (<a href="https://issues.apache.org/jira/browse/HBASE-11658">HBASE-11658)</a>.
Non-interactive mode captures the exit status (success or failure) of HBase Shell commands and passes that status back to the command interpreter.
If you use the normal interactive mode, the HBase Shell will only ever return its own exit status, which will nearly always be <code>0</code> for success.</p>
</div>
<div class="paragraph">
<p>To invoke non-interactive mode, pass the <code>-n</code> or <code>--non-interactive</code> option to HBase Shell.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.shell.noninteractive"><a class="anchor" href="#hbase.shell.noninteractive"></a>16. HBase Shell in OS Scripts</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can use the HBase shell from within operating system script interpreters like the Bash shell which is the default command interpreter for most Linux and UNIX distributions.
The following guidelines use Bash syntax, but could be adjusted to work with C-style shells such as csh or tcsh, and could probably be modified to work with the Microsoft Windows script interpreter as well. Submissions are welcome.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Spawning HBase Shell commands in this way is slow, so keep that in mind when you are deciding when combining HBase operations with the operating system command line is appropriate.
</td>
</tr>
</table>
</div>
<div class="exampleblock">
<div class="title">Example 4. Passing Commands to the HBase Shell</div>
<div class="content">
<div class="paragraph">
<p>You can pass commands to the HBase Shell in non-interactive mode (see <a href="#hbase.shell.noninteractive">hbase.shell.noninteractive</a>) using the <code>echo</code> command and the <code>|</code> (pipe) operator.
Be sure to escape characters in the HBase commands which would otherwise be interpreted by the shell.
Some debug-level output has been truncated from the example below.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ echo &quot;describe 'test1'&quot; | ./hbase shell -n
Version 0.98.3-hadoop2, rd5e65a9144e315bb0a964e7730871af32f5018d5, Sat May 31 19:56:09 PDT 2014
describe 'test1'
DESCRIPTION ENABLED
'test1', {NAME =&gt; 'cf', DATA_BLOCK_ENCODING =&gt; 'NON true
E', BLOOMFILTER =&gt; 'ROW', REPLICATION_SCOPE =&gt; '0',
VERSIONS =&gt; '1', COMPRESSION =&gt; 'NONE', MIN_VERSIO
NS =&gt; '0', TTL =&gt; 'FOREVER', KEEP_DELETED_CELLS =&gt;
'false', BLOCKSIZE =&gt; '65536', IN_MEMORY =&gt; 'false'
, BLOCKCACHE =&gt; 'true'}
1 row(s) in 3.2410 seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>To suppress all output, echo it to <em>/dev/null:</em></p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ echo &quot;describe 'test'&quot; | ./hbase shell -n &gt; /dev/null 2&gt;&amp;1</code></pre>
</div>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 5. Checking the Result of a Scripted Command</div>
<div class="content">
<div class="paragraph">
<p>Since scripts are not designed to be run interactively, you need a way to check whether your command failed or succeeded.
The HBase shell uses the standard convention of returning a value of <code>0</code> for successful commands, and some non-zero value for failed commands.
Bash stores a command&#8217;s return value in a special environment variable called <code>$?</code>.
Because that variable is overwritten each time the shell runs any command, you should store the result in a different, script-defined variable.</p>
</div>
<div class="paragraph">
<p>This is a naive script that shows one way to store the return value and make a decision based upon it.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">#!/bin/bash
echo &quot;describe 'test'&quot; | ./hbase shell -n &gt; /dev/null 2&gt;&amp;1
status=$?
echo &quot;The status was &quot; $status
if ($status == 0); then
echo &quot;The command succeeded&quot;
else
echo &quot;The command may have failed.&quot;
fi
return $status</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_checking_for_success_or_failure_in_scripts"><a class="anchor" href="#_checking_for_success_or_failure_in_scripts"></a>16.1. Checking for Success or Failure In Scripts</h3>
<div class="paragraph">
<p>Getting an exit code of <code>0</code> means that the command you scripted definitely succeeded.
However, getting a non-zero exit code does not necessarily mean the command failed.
The command could have succeeded, but the client lost connectivity, or some other event obscured its success.
This is because RPC commands are stateless.
The only way to be sure of the status of an operation is to check.
For instance, if your script creates a table, but returns a non-zero exit value, you should check whether the table was actually created before trying again to create it.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_read_hbase_shell_commands_from_a_command_file"><a class="anchor" href="#_read_hbase_shell_commands_from_a_command_file"></a>17. Read HBase Shell Commands from a Command File</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can enter HBase Shell commands into a text file, one command per line, and pass that file to the HBase Shell.</p>
</div>
<div class="listingblock">
<div class="title">Example Command File</div>
<div class="content">
<pre>create 'test', 'cf'
list 'test'
put 'test', 'row1', 'cf:a', 'value1'
put 'test', 'row2', 'cf:b', 'value2'
put 'test', 'row3', 'cf:c', 'value3'
put 'test', 'row4', 'cf:d', 'value4'
scan 'test'
get 'test', 'row1'
disable 'test'
enable 'test'</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 6. Directing HBase Shell to Execute the Commands</div>
<div class="content">
<div class="paragraph">
<p>Pass the path to the command file as the only argument to the <code>hbase shell</code> command.
Each command is executed and its output is shown.
If you do not include the <code>exit</code> command in your script, you are returned to the HBase shell prompt.
There is no way to programmatically check each individual command for success or failure.
Also, though you see the output for each command, the commands themselves are not echoed to the screen so it can be difficult to line up the command with its output.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./hbase shell ./sample_commands.txt
0 row(s) in 3.4170 seconds
TABLE
test
1 row(s) in 0.0590 seconds
0 row(s) in 0.1540 seconds
0 row(s) in 0.0080 seconds
0 row(s) in 0.0060 seconds
0 row(s) in 0.0060 seconds
ROW COLUMN+CELL
row1 column=cf:a, timestamp=1407130286968, value=value1
row2 column=cf:b, timestamp=1407130286997, value=value2
row3 column=cf:c, timestamp=1407130287007, value=value3
row4 column=cf:d, timestamp=1407130287015, value=value4
4 row(s) in 0.0420 seconds
COLUMN CELL
cf:a timestamp=1407130286968, value=value1
1 row(s) in 0.0110 seconds
0 row(s) in 1.5630 seconds
0 row(s) in 0.4360 seconds</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_passing_vm_options_to_the_shell"><a class="anchor" href="#_passing_vm_options_to_the_shell"></a>18. Passing VM Options to the Shell</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can pass VM options to the HBase Shell using the <code>HBASE_SHELL_OPTS</code> environment variable.
You can set this in your environment, for instance by editing <em>~/.bashrc</em>, or set it as part of the command to launch HBase Shell.
The following example sets several garbage-collection-related variables, just for the lifetime of the VM running the HBase Shell.
The command should be run all on a single line, but is broken by the <code>\</code> character, for readability.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HBASE_SHELL_OPTS=&quot;-verbose:gc -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCDateStamps \
-XX:+PrintGCDetails -Xloggc:$HBASE_HOME/logs/gc-hbase.log&quot; ./bin/hbase shell</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_overriding_configuration_starting_the_hbase_shell"><a class="anchor" href="#_overriding_configuration_starting_the_hbase_shell"></a>19. Overriding configuration starting the HBase Shell</h2>
<div class="sectionbody">
<div class="paragraph">
<p>As of hbase-2.0.5/hbase-2.1.3/hbase-2.2.0/hbase-1.4.10/hbase-1.5.0, you can
pass or override hbase configuration as specified in <code>hbase-*.xml</code> by passing
your key/values prefixed with <code>-D</code> on the command-line as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./bin/hbase shell -Dhbase.zookeeper.quorum=ZK0.remote.cluster.example.org,ZK1.remote.cluster.example.org,ZK2.remote.cluster.example.org -Draining=false
...
hbase(main):001:0&gt; @shell.hbase.configuration.get(&quot;hbase.zookeeper.quorum&quot;)
=&gt; &quot;ZK0.remote.cluster.example.org,ZK1.remote.cluster.example.org,ZK2.remote.cluster.example.org&quot;
hbase(main):002:0&gt; @shell.hbase.configuration.get(&quot;raining&quot;)
=&gt; &quot;false&quot;</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_shell_tricks"><a class="anchor" href="#_shell_tricks"></a>20. Shell Tricks</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_table_variables"><a class="anchor" href="#_table_variables"></a>20.1. Table variables</h3>
<div class="paragraph">
<p>HBase 0.95 adds shell commands that provides jruby-style object-oriented references for tables.
Previously all of the shell commands that act upon a table have a procedural style that always took the name of the table as an argument.
HBase 0.95 introduces the ability to assign a table to a jruby variable.
The table reference can be used to perform data read write operations such as puts, scans, and gets well as admin functionality such as disabling, dropping, describing tables.</p>
</div>
<div class="paragraph">
<p>For example, previously you would always specify a table name:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):000:0&gt; create 't', 'f'
0 row(s) in 1.0970 seconds
hbase(main):001:0&gt; put 't', 'rold', 'f', 'v'
0 row(s) in 0.0080 seconds
hbase(main):002:0&gt; scan 't'
ROW COLUMN+CELL
rold column=f:, timestamp=1378473207660, value=v
1 row(s) in 0.0130 seconds
hbase(main):003:0&gt; describe 't'
DESCRIPTION ENABLED
't', {NAME =&gt; 'f', DATA_BLOCK_ENCODING =&gt; 'NONE', BLOOMFILTER =&gt; 'ROW', REPLICATION_ true
SCOPE =&gt; '0', VERSIONS =&gt; '1', COMPRESSION =&gt; 'NONE', MIN_VERSIONS =&gt; '0', TTL =&gt; '2
147483647', KEEP_DELETED_CELLS =&gt; 'false', BLOCKSIZE =&gt; '65536', IN_MEMORY =&gt; 'false
', BLOCKCACHE =&gt; 'true'}
1 row(s) in 1.4430 seconds
hbase(main):004:0&gt; disable 't'
0 row(s) in 14.8700 seconds
hbase(main):005:0&gt; drop 't'
0 row(s) in 23.1670 seconds
hbase(main):006:0&gt;</pre>
</div>
</div>
<div class="paragraph">
<p>Now you can assign the table to a variable and use the results in jruby shell code.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):007 &gt; t = create 't', 'f'
0 row(s) in 1.0970 seconds
=&gt; Hbase::Table - t
hbase(main):008 &gt; t.put 'r', 'f', 'v'
0 row(s) in 0.0640 seconds
hbase(main):009 &gt; t.scan
ROW COLUMN+CELL
r column=f:, timestamp=1331865816290, value=v
1 row(s) in 0.0110 seconds
hbase(main):010:0&gt; t.describe
DESCRIPTION ENABLED
't', {NAME =&gt; 'f', DATA_BLOCK_ENCODING =&gt; 'NONE', BLOOMFILTER =&gt; 'ROW', REPLICATION_ true
SCOPE =&gt; '0', VERSIONS =&gt; '1', COMPRESSION =&gt; 'NONE', MIN_VERSIONS =&gt; '0', TTL =&gt; '2
147483647', KEEP_DELETED_CELLS =&gt; 'false', BLOCKSIZE =&gt; '65536', IN_MEMORY =&gt; 'false
', BLOCKCACHE =&gt; 'true'}
1 row(s) in 0.0210 seconds
hbase(main):038:0&gt; t.disable
0 row(s) in 6.2350 seconds
hbase(main):039:0&gt; t.drop
0 row(s) in 0.2340 seconds</pre>
</div>
</div>
<div class="paragraph">
<p>If the table has already been created, you can assign a Table to a variable by using the get_table method:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):011 &gt; create 't','f'
0 row(s) in 1.2500 seconds
=&gt; Hbase::Table - t
hbase(main):012:0&gt; tab = get_table 't'
0 row(s) in 0.0010 seconds
=&gt; Hbase::Table - t
hbase(main):013:0&gt; tab.put 'r1' ,'f', 'v'
0 row(s) in 0.0100 seconds
hbase(main):014:0&gt; tab.scan
ROW COLUMN+CELL
r1 column=f:, timestamp=1378473876949, value=v
1 row(s) in 0.0240 seconds
hbase(main):015:0&gt;</pre>
</div>
</div>
<div class="paragraph">
<p>The list functionality has also been extended so that it returns a list of table names as strings.
You can then use jruby to script table operations based on these names.
The list_snapshots command also acts similarly.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):016 &gt; tables = list('t.*')
TABLE
t
1 row(s) in 0.1040 seconds
=&gt; #&lt;#&lt;Class:0x7677ce29&gt;:0x21d377a4&gt;
hbase(main):017:0&gt; tables.map { |t| disable t ; drop t}
0 row(s) in 2.2510 seconds
=&gt; [nil]
hbase(main):018:0&gt;</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="irbrc"><a class="anchor" href="#irbrc"></a>20.2. <em>irbrc</em></h3>
<div class="paragraph">
<p>Create an <em>.irbrc</em> file for yourself in your home directory.
Add customizations.
A useful one is command history so commands are save across Shell invocations:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ more .irbrc
require 'irb/ext/save-history'
IRB.conf[:SAVE_HISTORY] = 100
IRB.conf[:HISTORY_FILE] = &quot;#{ENV['HOME']}/.irb-save-history&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you&#8217;d like to avoid printing the result of evaluting each expression to stderr, for example the array of tables returned from the "list" command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ echo &quot;IRB.conf[:ECHO] = false&quot; &gt;&gt;~/.irbrc</code></pre>
</div>
</div>
<div class="paragraph">
<p>See the <code>ruby</code> documentation of <em>.irbrc</em> to learn about other possible configurations.</p>
</div>
</div>
<div class="sect2">
<h3 id="_log_data_to_timestamp"><a class="anchor" href="#_log_data_to_timestamp"></a>20.3. LOG data to timestamp</h3>
<div class="paragraph">
<p>To convert the date '08/08/16 20:56:29' from an hbase log into a timestamp, do:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):021:0&gt; import java.text.SimpleDateFormat
hbase(main):022:0&gt; import java.text.ParsePosition
hbase(main):023:0&gt; SimpleDateFormat.new("yy/MM/dd HH:mm:ss").parse("08/08/16 20:56:29", ParsePosition.new(0)).getTime() =&gt; 1218920189000</pre>
</div>
</div>
<div class="paragraph">
<p>To go the other direction:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):021:0&gt; import java.util.Date
hbase(main):022:0&gt; Date.new(1218920189000).toString() =&gt; "Sat Aug 16 20:56:29 UTC 2008"</pre>
</div>
</div>
<div class="paragraph">
<p>To output in a format that is exactly like that of the HBase log format will take a little messing with <a href="http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html">SimpleDateFormat</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_query_shell_configuration"><a class="anchor" href="#_query_shell_configuration"></a>20.4. Query Shell Configuration</h3>
<div class="listingblock">
<div class="content">
<pre>hbase(main):001:0&gt; @shell.hbase.configuration.get("hbase.rpc.timeout")
=&gt; "60000"</pre>
</div>
</div>
<div class="paragraph">
<p>To set a config in the shell:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):005:0&gt; @shell.hbase.configuration.setInt("hbase.rpc.timeout", 61010)
hbase(main):006:0&gt; @shell.hbase.configuration.get("hbase.rpc.timeout")
=&gt; "61010"</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="tricks.pre-split"><a class="anchor" href="#tricks.pre-split"></a>20.5. Pre-splitting tables with the HBase Shell</h3>
<div class="paragraph">
<p>You can use a variety of options to pre-split tables when creating them via the HBase Shell <code>create</code> command.</p>
</div>
<div class="paragraph">
<p>The simplest approach is to specify an array of split points when creating the table. Note that when specifying string literals as split points, these will create split points based on the underlying byte representation of the string. So when specifying a split point of '10', we are actually specifying the byte split point '\x31\30'.</p>
</div>
<div class="paragraph">
<p>The split points will define <code>n+1</code> regions where <code>n</code> is the number of split points. The lowest region will contain all keys from the lowest possible key up to but not including the first split point key.
The next region will contain keys from the first split point up to, but not including the next split point key.
This will continue for all split points up to the last. The last region will be defined from the last split point up to the maximum possible key.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f</span><span class="delimiter">'</span></span>,SPLITS =&gt; [<span class="string"><span class="delimiter">'</span><span class="content">10</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">20</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">30</span><span class="delimiter">'</span></span>]</code></pre>
</div>
</div>
<div class="paragraph">
<p>In the above example, the table 't1' will be created with column family 'f', pre-split to four regions. Note the first region will contain all keys from '\x00' up to '\x30' (as '\x31' is the ASCII code for '1').</p>
</div>
<div class="paragraph">
<p>You can pass the split points in a file using following variation. In this example, the splits are read from a file corresponding to the local path on the local filesystem. Each line in the file specifies a split point key.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t14</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f</span><span class="delimiter">'</span></span>,SPLITS_FILE=&gt;<span class="string"><span class="delimiter">'</span><span class="content">splits.txt</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The other options are to automatically compute splits based on a desired number of regions and a splitting algorithm.
HBase supplies algorithms for splitting the key range based on uniform splits or based on hexadecimal keys, but you can provide your own splitting algorithm to subdivide the key range.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">#</span> create table with four regions based on random bytes keys
hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t2</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>, { NUMREGIONS =&gt; <span class="integer">4</span> , SPLITALGO =&gt; <span class="string"><span class="delimiter">'</span><span class="content">UniformSplit</span><span class="delimiter">'</span></span> }
<span class="error">#</span> create table with five regions based on hex keys
hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t3</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>, { NUMREGIONS =&gt; <span class="integer">5</span>, SPLITALGO =&gt; <span class="string"><span class="delimiter">'</span><span class="content">HexStringSplit</span><span class="delimiter">'</span></span> }</code></pre>
</div>
</div>
<div class="paragraph">
<p>As the HBase Shell is effectively a Ruby environment, you can use simple Ruby scripts to compute splits algorithmically.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">#</span> generate splits <span class="keyword">for</span> <span class="type">long</span> (Ruby fixnum) key range from start to end key
hbase(main):<span class="octal">070</span>:<span class="integer">0</span>&gt; def gen_splits(start_key,end_key,num_regions)
hbase(main):<span class="octal">071</span>:<span class="integer">1</span>&gt; results=<span class="type">[]</span>
hbase(main):<span class="octal">072</span>:<span class="integer">1</span>&gt; range=end_key-start_key
hbase(main):<span class="octal">073</span>:<span class="integer">1</span>&gt; incr=(range/num_regions).floor
hbase(main):<span class="octal">074</span>:<span class="integer">1</span>&gt; <span class="keyword">for</span> i in <span class="integer">1</span> .. num_regions-<span class="integer">1</span>
hbase(main):<span class="octal">075</span>:<span class="integer">2</span>&gt; results.push([i*incr+start_key].pack(<span class="string"><span class="delimiter">&quot;</span><span class="content">N</span><span class="delimiter">&quot;</span></span>))
hbase(main):<span class="octal">076</span>:<span class="integer">2</span>&gt; end
hbase(main):<span class="octal">077</span>:<span class="integer">1</span>&gt; <span class="keyword">return</span> results
hbase(main):<span class="integer">078</span>:<span class="integer">1</span>&gt; end
hbase(main):<span class="integer">079</span>:<span class="integer">0</span>&gt;
hbase(main):<span class="integer">080</span>:<span class="integer">0</span>&gt; splits=gen_splits(<span class="integer">1</span>,<span class="integer">2000000</span>,<span class="integer">10</span>)
=&gt; [<span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\003</span><span class="char">\r</span><span class="content">@</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\006</span><span class="char">\032</span><span class="char">\177</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\t</span><span class="content">'</span><span class="char">\276</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\f</span><span class="content">4</span><span class="char">\375</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\017</span><span class="content">B&lt;</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\022</span><span class="content">O{</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\025</span><span class="char">\\</span><span class="char">\272</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="char">\030</span><span class="content">i</span><span class="char">\371</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="char">\000</span><span class="content">\e</span><span class="content">w8</span><span class="delimiter">&quot;</span></span>]
hbase(main):<span class="integer">081</span>:<span class="integer">0</span>&gt; create <span class="string"><span class="delimiter">'</span><span class="content">test_splits</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f</span><span class="delimiter">'</span></span>,SPLITS=&gt;splits
<span class="integer">0</span> row(s) in <span class="float">0.2670</span> seconds
=&gt; Hbase::Table - test_splits</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note that the HBase Shell command <code>truncate</code> effectively drops and recreates the table with default options which will discard any pre-splitting.
If you need to truncate a pre-split table, you must drop and recreate the table explicitly to re-specify custom split options.</p>
</div>
</div>
<div class="sect2">
<h3 id="_debug"><a class="anchor" href="#_debug"></a>20.6. Debug</h3>
<div class="sect3">
<h4 id="_shell_debug_switch"><a class="anchor" href="#_shell_debug_switch"></a>20.6.1. Shell debug switch</h4>
<div class="paragraph">
<p>You can set a debug switch in the shell to see more output&#8201;&#8212;&#8201;e.g.
more of the stack trace on exception&#8201;&#8212;&#8201;when you run a command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; debug &lt;RETURN&gt;</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_debug_log_level"><a class="anchor" href="#_debug_log_level"></a>20.6.2. DEBUG log level</h4>
<div class="paragraph">
<p>To enable DEBUG level logging in the shell, launch it with the <code>-d</code> option.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./bin/hbase shell -d</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_commands"><a class="anchor" href="#_commands"></a>20.7. Commands</h3>
<div class="sect3">
<h4 id="_count"><a class="anchor" href="#_count"></a>20.7.1. count</h4>
<div class="paragraph">
<p>Count command returns the number of rows in a table.
It&#8217;s quite fast when configured with the right CACHE</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; count <span class="string"><span class="delimiter">'</span><span class="content">&lt;tablename&gt;</span><span class="delimiter">'</span></span>, CACHE =&gt; <span class="integer">1000</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The above count fetches 1000 rows at a time.
Set CACHE lower if your rows are big.
Default is to fetch one row at a time.</p>
</div>
</div>
</div>
</div>
</div>
<h1 id="datamodel" class="sect0"><a class="anchor" href="#datamodel"></a>Data Model</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>In HBase, data is stored in tables, which have rows and columns.
This is a terminology overlap with relational databases (RDBMSs), but this is not a helpful analogy.
Instead, it can be helpful to think of an HBase table as a multi-dimensional map.</p>
</div>
<div class="dlist">
<div class="title">HBase Data Model Terminology</div>
<dl>
<dt class="hdlist1">Table</dt>
<dd>
<p>An HBase table consists of multiple rows.</p>
</dd>
<dt class="hdlist1">Row</dt>
<dd>
<p>A row in HBase consists of a row key and one or more columns with values associated with them.
Rows are sorted alphabetically by the row key as they are stored.
For this reason, the design of the row key is very important.
The goal is to store data in such a way that related rows are near each other.
A common row key pattern is a website domain.
If your row keys are domains, you should probably store them in reverse (org.apache.www, org.apache.mail, org.apache.jira). This way, all of the Apache domains are near each other in the table, rather than being spread out based on the first letter of the subdomain.</p>
</dd>
<dt class="hdlist1">Column</dt>
<dd>
<p>A column in HBase consists of a column family and a column qualifier, which are delimited by a <code>:</code> (colon) character.</p>
</dd>
<dt class="hdlist1">Column Family</dt>
<dd>
<p>Column families physically colocate a set of columns and their values, often for performance reasons.
Each column family has a set of storage properties, such as whether its values should be cached in memory, how its data is compressed or its row keys are encoded, and others.
Each row in a table has the same column families, though a given row might not store anything in a given column family.</p>
</dd>
<dt class="hdlist1">Column Qualifier</dt>
<dd>
<p>A column qualifier is added to a column family to provide the index for a given piece of data.
Given a column family <code>content</code>, a column qualifier might be <code>content:html</code>, and another might be <code>content:pdf</code>.
Though column families are fixed at table creation, column qualifiers are mutable and may differ greatly between rows.</p>
</dd>
<dt class="hdlist1">Cell</dt>
<dd>
<p>A cell is a combination of row, column family, and column qualifier, and contains a value and a timestamp, which represents the value&#8217;s version.</p>
</dd>
<dt class="hdlist1">Timestamp</dt>
<dd>
<p>A timestamp is written alongside each value, and is the identifier for a given version of a value.
By default, the timestamp represents the time on the RegionServer when the data was written, but you can specify a different timestamp value when you put data into the cell.</p>
</dd>
</dl>
</div>
</div>
</div>
<div class="sect1">
<h2 id="conceptual.view"><a class="anchor" href="#conceptual.view"></a>21. Conceptual View</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can read a very understandable explanation of the HBase data model in the blog post <a href="https://dzone.com/articles/understanding-hbase-and-bigtab">Understanding HBase and BigTable</a> by Jim R. Wilson.
Another good explanation is available in the PDF <a href="http://0b4af6cdc2f0c5998459-c0245c5c937c5dedcca3f1764ecc9b2f.r43.cf2.rackcdn.com/9353-login1210_khurana.pdf">Introduction to Basic Schema Design</a> by Amandeep Khurana.</p>
</div>
<div class="paragraph">
<p>It may help to read different perspectives to get a solid understanding of HBase schema design.
The linked articles cover the same ground as the information in this section.</p>
</div>
<div class="paragraph">
<p>The following example is a slightly modified form of the one on page 2 of the <a href="http://research.google.com/archive/bigtable.html">BigTable</a> paper.
There is a table called <code>webtable</code> that contains two rows (<code>com.cnn.www</code> and <code>com.example.www</code>) and three column families named <code>contents</code>, <code>anchor</code>, and <code>people</code>.
In this example, for the first row (<code>com.cnn.www</code>), <code>anchor</code> contains two columns (<code>anchor:cssnsi.com</code>, <code>anchor:my.look.ca</code>) and <code>contents</code> contains one column (<code>contents:html</code>). This example contains 5 versions of the row with the row key <code>com.cnn.www</code>, and one version of the row with the row key <code>com.example.www</code>.
The <code>contents:html</code> column qualifier contains the entire HTML of a given website.
Qualifiers of the <code>anchor</code> column family each contain the external site which links to the site represented by the row, along with the text it used in the anchor of its link.
The <code>people</code> column family represents people associated with the site.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Column Names</div>
<div class="paragraph">
<p>By convention, a column name is made of its column family prefix and a <em>qualifier</em>.
For example, the column <em>contents:html</em> is made up of the column family <code>contents</code> and the <code>html</code> qualifier.
The colon character (<code>:</code>) delimits the column family from the column family <em>qualifier</em>.</p>
</div>
</td>
</tr>
</table>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 6. Table <code>webtable</code></caption>
<colgroup>
<col style="width: 20%;">
<col style="width: 20%;">
<col style="width: 20%;">
<col style="width: 20%;">
<col style="width: 20%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Row Key</th>
<th class="tableblock halign-left valign-top">Time Stamp</th>
<th class="tableblock halign-left valign-top">ColumnFamily <code>contents</code></th>
<th class="tableblock halign-left valign-top">ColumnFamily <code>anchor</code></th>
<th class="tableblock halign-left valign-top">ColumnFamily <code>people</code></th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t9</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">anchor:cnnsi.com = "CNN"</p></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t8</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">anchor:my.look.ca = "CNN.com"</p></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t6</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t5</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t3</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.example.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t5</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">people:author = "John Doe"</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Cells in this table that appear to be empty do not take space, or in fact exist, in HBase.
This is what makes HBase "sparse." A tabular view is not the only possible way to look at data in HBase, or even the most accurate.
The following represents the same information as a multi-dimensional map.
This is only a mock-up for illustrative purposes and may not be strictly accurate.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="json">{
<span class="key"><span class="delimiter">&quot;</span><span class="content">com.cnn.www</span><span class="delimiter">&quot;</span></span>: {
<span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>: {
<span class="error">t</span><span class="integer">6</span>: <span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>:<span class="error">h</span><span class="error">t</span><span class="error">m</span><span class="error">l</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">&lt;html&gt;...</span><span class="delimiter">&quot;</span></span>
<span class="error">t</span><span class="integer">5</span>: <span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>:<span class="error">h</span><span class="error">t</span><span class="error">m</span><span class="error">l</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">&lt;html&gt;...</span><span class="delimiter">&quot;</span></span>
<span class="error">t</span><span class="integer">3</span>: <span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>:<span class="error">h</span><span class="error">t</span><span class="error">m</span><span class="error">l</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">&lt;html&gt;...</span><span class="delimiter">&quot;</span></span>
}
<span class="error">a</span><span class="error">n</span><span class="error">c</span><span class="error">h</span><span class="error">o</span><span class="error">r</span>: {
<span class="error">t</span><span class="integer">9</span>: <span class="error">a</span><span class="error">n</span><span class="error">c</span><span class="error">h</span><span class="error">o</span><span class="error">r</span>:<span class="error">c</span><span class="error">n</span><span class="error">n</span><span class="error">s</span><span class="error">i</span><span class="error">.</span><span class="error">c</span><span class="error">o</span><span class="error">m</span> <span class="error">=</span> <span class="string"><span class="delimiter">&quot;</span><span class="content">CNN</span><span class="delimiter">&quot;</span></span>
<span class="error">t</span><span class="integer">8</span>: <span class="error">a</span><span class="error">n</span><span class="error">c</span><span class="error">h</span><span class="error">o</span><span class="error">r</span>:<span class="error">m</span><span class="error">y</span><span class="error">.</span><span class="error">l</span><span class="error">o</span><span class="error">o</span><span class="error">k</span><span class="error">.</span><span class="error">c</span><span class="error">a</span> <span class="error">=</span> <span class="string"><span class="delimiter">&quot;</span><span class="content">CNN.com</span><span class="delimiter">&quot;</span></span>
}
<span class="error">p</span><span class="error">e</span><span class="error">o</span><span class="error">p</span><span class="error">l</span><span class="error">e</span>: {}
}
<span class="key"><span class="delimiter">&quot;</span><span class="content">com.example.www</span><span class="delimiter">&quot;</span></span>: {
<span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>: {
<span class="error">t</span><span class="integer">5</span>: <span class="error">c</span><span class="error">o</span><span class="error">n</span><span class="error">t</span><span class="error">e</span><span class="error">n</span><span class="error">t</span><span class="error">s</span>:<span class="error">h</span><span class="error">t</span><span class="error">m</span><span class="error">l</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">&lt;html&gt;...</span><span class="delimiter">&quot;</span></span>
}
<span class="error">a</span><span class="error">n</span><span class="error">c</span><span class="error">h</span><span class="error">o</span><span class="error">r</span>: {}
<span class="error">p</span><span class="error">e</span><span class="error">o</span><span class="error">p</span><span class="error">l</span><span class="error">e</span>: {
<span class="error">t</span><span class="integer">5</span>: <span class="error">p</span><span class="error">e</span><span class="error">o</span><span class="error">p</span><span class="error">l</span><span class="error">e</span>:<span class="error">a</span><span class="error">u</span><span class="error">t</span><span class="error">h</span><span class="error">o</span><span class="error">r</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">John Doe</span><span class="delimiter">&quot;</span></span>
}
}
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="physical.view"><a class="anchor" href="#physical.view"></a>22. Physical View</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Although at a conceptual level tables may be viewed as a sparse set of rows, they are physically stored by column family.
A new column qualifier (column_family:column_qualifier) can be added to an existing column family at any time.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 7. ColumnFamily <code>anchor</code></caption>
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Row Key</th>
<th class="tableblock halign-left valign-top">Time Stamp</th>
<th class="tableblock halign-left valign-top">Column Family <code>anchor</code></th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t9</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>anchor:cnnsi.com = "CNN"</code></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t8</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>anchor:my.look.ca = "CNN.com"</code></p></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 8. ColumnFamily <code>contents</code></caption>
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Row Key</th>
<th class="tableblock halign-left valign-top">Time Stamp</th>
<th class="tableblock halign-left valign-top">ColumnFamily <code>contents:</code></th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t6</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t5</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">"com.cnn.www"</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">t3</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contents:html = "&lt;html&gt;&#8230;&#8203;"</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>The empty cells shown in the conceptual view are not stored at all.
Thus a request for the value of the <code>contents:html</code> column at time stamp <code>t8</code> would return no value.
Similarly, a request for an <code>anchor:my.look.ca</code> value at time stamp <code>t9</code> would return no value.
However, if no timestamp is supplied, the most recent value for a particular column would be returned.
Given multiple versions, the most recent is also the first one found, since timestamps are stored in descending order.
Thus a request for the values of all columns in the row <code>com.cnn.www</code> if no timestamp is specified would be: the value of <code>contents:html</code> from timestamp <code>t6</code>, the value of <code>anchor:cnnsi.com</code> from timestamp <code>t9</code>, the value of <code>anchor:my.look.ca</code> from timestamp <code>t8</code>.</p>
</div>
<div class="paragraph">
<p>For more information about the internals of how Apache HBase stores data, see <a href="#regions.arch">regions.arch</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_namespace"><a class="anchor" href="#_namespace"></a>23. Namespace</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A namespace is a logical grouping of tables analogous to a database in relation database systems.
This abstraction lays the groundwork for upcoming multi-tenancy related features:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Quota Management (<a href="https://issues.apache.org/jira/browse/HBASE-8410">HBASE-8410</a>) - Restrict the amount of resources (i.e. regions, tables) a namespace can consume.</p>
</li>
<li>
<p>Namespace Security Administration (<a href="https://issues.apache.org/jira/browse/HBASE-9206">HBASE-9206</a>) - Provide another level of security administration for tenants.</p>
</li>
<li>
<p>Region server groups (<a href="https://issues.apache.org/jira/browse/HBASE-6721">HBASE-6721</a>) - A namespace/table can be pinned onto a subset of RegionServers thus guaranteeing a coarse level of isolation.</p>
</li>
</ul>
</div>
<div class="sect2">
<h3 id="namespace_creation"><a class="anchor" href="#namespace_creation"></a>23.1. Namespace management</h3>
<div class="paragraph">
<p>A namespace can be created, removed or altered.
Namespace membership is determined during table creation by specifying a fully-qualified table name of the form:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;table</span> <span class="attribute-name">namespace</span><span class="tag">&gt;</span>:<span class="tag">&lt;table</span> <span class="attribute-name">qualifier</span><span class="tag">&gt;</span></code></pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 7. Examples</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">#Create a namespace
create_namespace 'my_ns'</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">#create my_table in my_ns namespace
create 'my_ns:my_table', 'fam'</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">#drop namespace
drop_namespace 'my_ns'</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">#alter namespace
alter_namespace 'my_ns', {METHOD =&gt; 'set', 'PROPERTY_NAME' =&gt; 'PROPERTY_VALUE'}</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="namespace_special"><a class="anchor" href="#namespace_special"></a>23.2. Predefined namespaces</h3>
<div class="paragraph">
<p>There are two predefined special namespaces:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>hbase - system namespace, used to contain HBase internal tables</p>
</li>
<li>
<p>default - tables with no explicit specified namespace will automatically fall into this namespace</p>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 8. Examples</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">#namespace=foo and table qualifier=bar
create 'foo:bar', 'fam'
#namespace=default and table qualifier=bar
create 'bar', 'fam'</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_table"><a class="anchor" href="#_table"></a>24. Table</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Tables are declared up front at schema definition time.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_row"><a class="anchor" href="#_row"></a>25. Row</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Row keys are uninterpreted bytes.
Rows are lexicographically sorted with the lowest order appearing first in a table.
The empty byte array is used to denote both the start and end of a tables' namespace.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="columnfamily"><a class="anchor" href="#columnfamily"></a>26. Column Family</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Columns in Apache HBase are grouped into <em>column families</em>.
All column members of a column family have the same prefix.
For example, the columns <em>courses:history</em> and <em>courses:math</em> are both members of the <em>courses</em> column family.
The colon character (<code>:</code>) delimits the column family from the column family qualifier.
The column family prefix must be composed of <em>printable</em> characters.
The qualifying tail, the column family <em>qualifier</em>, can be made of any arbitrary bytes.
Column families must be declared up front at schema definition time whereas columns do not need to be defined at schema time but can be conjured on the fly while the table is up and running.</p>
</div>
<div class="paragraph">
<p>Physically, all column family members are stored together on the filesystem.
Because tunings and storage specifications are done at the column family level, it is advised that all column family members have the same general access pattern and size characteristics.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_cells"><a class="anchor" href="#_cells"></a>27. Cells</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A <em>{row, column, version}</em> tuple exactly specifies a <code>cell</code> in HBase.
Cell content is uninterpreted bytes</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_data_model_operations"><a class="anchor" href="#_data_model_operations"></a>28. Data Model Operations</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The four primary data model operations are Get, Put, Scan, and Delete.
Operations are applied via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html">Table</a> instances.</p>
</div>
<div class="sect2">
<h3 id="_get"><a class="anchor" href="#_get"></a>28.1. Get</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</a> returns attributes for a specified row.
Gets are executed via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#get-org.apache.hadoop.hbase.client.Get-">Table.get</a></p>
</div>
</div>
<div class="sect2">
<h3 id="_put"><a class="anchor" href="#_put"></a>28.2. Put</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html">Put</a> either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#put-org.apache.hadoop.hbase.client.Put-">Table.put</a> (non-writeBuffer) or <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch-java.util.List-java.lang.Object:A-">Table.batch</a> (non-writeBuffer)</p>
</div>
</div>
<div class="sect2">
<h3 id="scan"><a class="anchor" href="#scan"></a>28.3. Scans</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> allow iteration over multiple rows for specified attributes.</p>
</div>
<div class="paragraph">
<p>The following is an example of a Scan on a Table instance.
Assume that a table is populated with rows with keys "row1", "row2", "row3", and then another set of rows with the keys "abc1", "abc2", and "abc3". The following example shows how to set a Scan instance to return the rows beginning with "row".</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Table table = ... <span class="comment">// instantiate a Table instance</span>
Scan scan = <span class="keyword">new</span> Scan();
scan.addColumn(CF, ATTR);
scan.setRowPrefixFilter(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">row</span><span class="delimiter">&quot;</span></span>));
ResultScanner rs = table.getScanner(scan);
<span class="keyword">try</span> {
<span class="keyword">for</span> (<span class="predefined-type">Result</span> r = rs.next(); r != <span class="predefined-constant">null</span>; r = rs.next()) {
<span class="comment">// process result...</span>
}
} <span class="keyword">finally</span> {
rs.close(); <span class="comment">// always close the ResultScanner!</span>
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note that generally the easiest way to specify a specific stop point for a scan is by using the <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/InclusiveStopFilter.html">InclusiveStopFilter</a> class.</p>
</div>
</div>
<div class="sect2">
<h3 id="_delete"><a class="anchor" href="#_delete"></a>28.4. Delete</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Delete.html">Delete</a> removes a row from a table.
Deletes are executed via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete-org.apache.hadoop.hbase.client.Delete-">Table.delete</a>.</p>
</div>
<div class="paragraph">
<p>HBase does not modify data in place, and so deletes are handled by creating new markers called <em>tombstones</em>.
These tombstones, along with the dead values, are cleaned up on major compactions.</p>
</div>
<div class="paragraph">
<p>See <a href="#version.delete">version.delete</a> for more information on deleting versions of columns, and see <a href="#compaction">compaction</a> for more information on compactions.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="versions"><a class="anchor" href="#versions"></a>29. Versions</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A <em>{row, column, version}</em> tuple exactly specifies a <code>cell</code> in HBase.
It&#8217;s possible to have an unbounded number of cells where the row and column are the same but the cell address differs only in its version dimension.</p>
</div>
<div class="paragraph">
<p>While rows and column keys are expressed as bytes, the version is specified using a long integer.
Typically this long contains time instances such as those returned by <code>java.util.Date.getTime()</code> or <code>System.currentTimeMillis()</code>, that is: <em class="quote">the difference, measured in milliseconds, between the current time and midnight, January 1, 1970 UTC</em>.</p>
</div>
<div class="paragraph">
<p>The HBase version dimension is stored in decreasing order, so that when reading from a store file, the most recent values are found first.</p>
</div>
<div class="paragraph">
<p>There is a lot of confusion over the semantics of <code>cell</code> versions, in HBase.
In particular:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>If multiple writes to a cell have the same version, only the last written is fetchable.</p>
</li>
<li>
<p>It is OK to write cells in a non-increasing version order.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Below we describe how the version dimension in HBase currently works.
See <a href="https://issues.apache.org/jira/browse/HBASE-2406">HBASE-2406</a> for discussion of HBase versions. <a href="https://www.ngdata.com/bending-time-in-hbase/">Bending time in HBase</a> makes for a good read on the version, or time, dimension in HBase.
It has more detail on versioning than is provided here.</p>
</div>
<div class="paragraph">
<p>As of this writing, the limitation <em>Overwriting values at existing timestamps</em> mentioned in the article no longer holds in HBase.
This section is basically a synopsis of this article by Bruno Dumon.</p>
</div>
<div class="sect2">
<h3 id="specify.number.of.versions"><a class="anchor" href="#specify.number.of.versions"></a>29.1. Specifying the Number of Versions to Store</h3>
<div class="paragraph">
<p>The maximum number of versions to store for a given column is part of the column schema and is specified at table creation, or via an <code>alter</code> command, via <code>HColumnDescriptor.DEFAULT_VERSIONS</code>.
Prior to HBase 0.96, the default number of versions kept was <code>3</code>, but in 0.96 and newer has been changed to <code>1</code>.</p>
</div>
<div class="exampleblock">
<div class="title">Example 9. Modify the Maximum Number of Versions for a Column Family</div>
<div class="content">
<div class="paragraph">
<p>This example uses HBase Shell to keep a maximum of 5 versions of all columns in column family <code>f1</code>.
You could also use <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; alter ‘t1′, NAME =&gt; ‘f1′, VERSIONS =&gt; 5</pre>
</div>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 10. Modify the Minimum Number of Versions for a Column Family</div>
<div class="content">
<div class="paragraph">
<p>You can also specify the minimum number of versions to store per column family.
By default, this is set to 0, which means the feature is disabled.
The following example sets the minimum number of versions on all columns in column family <code>f1</code> to <code>2</code>, via HBase Shell.
You could also use <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; alter ‘t1′, NAME =&gt; ‘f1′, MIN_VERSIONS =&gt; 2</pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>Starting with HBase 0.98.2, you can specify a global default for the maximum number of versions kept for all newly-created columns, by setting <code>hbase.column.max.version</code> in <em>hbase-site.xml</em>.
See <a href="#hbase.column.max.version">hbase.column.max.version</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="versions.ops"><a class="anchor" href="#versions.ops"></a>29.2. Versions and HBase Operations</h3>
<div class="paragraph">
<p>In this section we look at the behavior of the version dimension for each of the core HBase operations.</p>
</div>
<div class="sect3">
<h4 id="_get_scan"><a class="anchor" href="#_get_scan"></a>29.2.1. Get/Scan</h4>
<div class="paragraph">
<p>Gets are implemented on top of Scans.
The below discussion of <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</a> applies equally to <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scans</a>.</p>
</div>
<div class="paragraph">
<p>By default, i.e. if you specify no explicit version, when doing a <code>get</code>, the cell whose version has the largest value is returned (which may or may not be the latest one written, see later). The default behavior can be modified in the following ways:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>to return more than one version, see <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html#setMaxVersions--">Get.setMaxVersions()</a></p>
</li>
<li>
<p>to return versions other than the latest, see <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html#setTimeRange-long-long-">Get.setTimeRange()</a></p>
<div class="paragraph">
<p>To retrieve the latest version that is less than or equal to a given value, thus giving the 'latest' state of the record at a certain point in time, just use a range from 0 to the desired version and set the max versions to 1.</p>
</div>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_default_get_example"><a class="anchor" href="#_default_get_example"></a>29.2.2. Default Get Example</h4>
<div class="paragraph">
<p>The following Get will only retrieve the current version of the row</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Get get = <span class="keyword">new</span> Get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">row1</span><span class="delimiter">&quot;</span></span>));
<span class="predefined-type">Result</span> r = table.get(get);
<span class="type">byte</span><span class="type">[]</span> b = r.getValue(CF, ATTR); <span class="comment">// returns current version of value</span></code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_versioned_get_example"><a class="anchor" href="#_versioned_get_example"></a>29.2.3. Versioned Get Example</h4>
<div class="paragraph">
<p>The following Get will return the last 3 versions of the row.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Get get = <span class="keyword">new</span> Get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">row1</span><span class="delimiter">&quot;</span></span>));
get.setMaxVersions(<span class="integer">3</span>); <span class="comment">// will return last 3 versions of row</span>
<span class="predefined-type">Result</span> r = table.get(get);
<span class="type">byte</span><span class="type">[]</span> b = r.getValue(CF, ATTR); <span class="comment">// returns current version of value</span>
<span class="predefined-type">List</span>&lt;Cell&gt; cells = r.getColumnCells(CF, ATTR); <span class="comment">// returns all versions of this column</span></code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_put_2"><a class="anchor" href="#_put_2"></a>29.2.4. Put</h4>
<div class="paragraph">
<p>Doing a put always creates a new version of a <code>cell</code>, at a certain timestamp.
By default the system uses the server&#8217;s <code>currentTimeMillis</code>, but you can specify the version (= the long integer) yourself, on a per-column level.
This means you could assign a time in the past or the future, or use the long value for non-time purposes.</p>
</div>
<div class="paragraph">
<p>To overwrite an existing value, do a put at exactly the same row, column, and version as that of the cell you want to overwrite.</p>
</div>
<div class="sect4">
<h5 id="_implicit_version_example"><a class="anchor" href="#_implicit_version_example"></a>Implicit Version Example</h5>
<div class="paragraph">
<p>The following Put will be implicitly versioned by HBase with the current time.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Put put = <span class="keyword">new</span> Put(Bytes.toBytes(row));
put.add(CF, ATTR, Bytes.toBytes( data));
table.put(put);</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="_explicit_version_example"><a class="anchor" href="#_explicit_version_example"></a>Explicit Version Example</h5>
<div class="paragraph">
<p>The following Put has the version timestamp explicitly set.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Put put = <span class="keyword">new</span> Put( Bytes.toBytes(row));
<span class="type">long</span> explicitTimeInMs = <span class="integer">555</span>; <span class="comment">// just an example</span>
put.add(CF, ATTR, explicitTimeInMs, Bytes.toBytes(data));
table.put(put);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Caution: the version timestamp is used internally by HBase for things like time-to-live calculations.
It&#8217;s usually best to avoid setting this timestamp yourself.
Prefer using a separate timestamp attribute of the row, or have the timestamp as a part of the row key, or both.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="version.delete"><a class="anchor" href="#version.delete"></a>29.2.5. Delete</h4>
<div class="paragraph">
<p>There are three different types of internal delete markers.
See Lars Hofhansl&#8217;s blog for discussion of his attempt adding another, <a href="http://hadoop-hbase.blogspot.com/2012/01/scanning-in-hbase.html">Scanning in HBase: Prefix Delete Marker</a>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Delete: for a specific version of a column.</p>
</li>
<li>
<p>Delete column: for all versions of a column.</p>
</li>
<li>
<p>Delete family: for all columns of a particular ColumnFamily</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>When deleting an entire row, HBase will internally create a tombstone for each ColumnFamily (i.e., not each individual column).</p>
</div>
<div class="paragraph">
<p>Deletes work by creating <em>tombstone</em> markers.
For example, let&#8217;s suppose we want to delete a row.
For this you can specify a version, or else by default the <code>currentTimeMillis</code> is used.
What this means is <em>delete all cells where the version is less than or equal to this version</em>.
HBase never modifies data in place, so for example a delete will not immediately delete (or mark as deleted) the entries in the storage file that correspond to the delete condition.
Rather, a so-called <em>tombstone</em> is written, which will mask the deleted values.
When HBase does a major compaction, the tombstones are processed to actually remove the dead values, together with the tombstones themselves.
If the version you specified when deleting a row is larger than the version of any value in the row, then you can consider the complete row to be deleted.</p>
</div>
<div class="paragraph">
<p>For an informative discussion on how deletes and versioning interact, see the thread <a href="http://comments.gmane.org/gmane.comp.java.hadoop.hbase.user/28421">Put w/timestamp &#8594; Deleteall &#8594; Put w/ timestamp fails</a> up on the user mailing list.</p>
</div>
<div class="paragraph">
<p>Also see <a href="#keyvalue">keyvalue</a> for more information on the internal KeyValue format.</p>
</div>
<div class="paragraph">
<p>Delete markers are purged during the next major compaction of the store, unless the <code>KEEP_DELETED_CELLS</code> option is set in the column family (See <a href="#cf.keep.deleted">Keeping Deleted Cells</a>).
To keep the deletes for a configurable amount of time, you can set the delete TTL via the hbase.hstore.time.to.purge.deletes property in <em>hbase-site.xml</em>.
If <code>hbase.hstore.time.to.purge.deletes</code> is not set, or set to 0, all delete markers, including those with timestamps in the future, are purged during the next major compaction.
Otherwise, a delete marker with a timestamp in the future is kept until the major compaction which occurs after the time represented by the marker&#8217;s timestamp plus the value of <code>hbase.hstore.time.to.purge.deletes</code>, in milliseconds.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
This behavior represents a fix for an unexpected change that was introduced in HBase 0.94, and was fixed in <a href="https://issues.apache.org/jira/browse/HBASE-10118">HBASE-10118</a>.
The change has been backported to HBase 0.94 and newer branches.
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="new.version.behavior"><a class="anchor" href="#new.version.behavior"></a>29.3. Optional New Version and Delete behavior in HBase-2.0.0</h3>
<div class="paragraph">
<p>In <code>hbase-2.0.0</code>, the operator can specify an alternate version and
delete treatment by setting the column descriptor property
<code>NEW_VERSION_BEHAVIOR</code> to true (To set a property on a column family
descriptor, you must first disable the table and then alter the
column family descriptor; see <a href="#cf.keep.deleted">Keeping Deleted Cells</a> for an example
of editing an attribute on a column family descriptor).</p>
</div>
<div class="paragraph">
<p>The 'new version behavior', undoes the limitations listed below
whereby a <code>Delete</code> ALWAYS overshadows a <code>Put</code> if at the same
location&#8201;&#8212;&#8201;i.e. same row, column family, qualifier and timestamp&#8201;&#8212;&#8201;regardless of which arrived first. Version accounting is also
changed as deleted versions are considered toward total version count.
This is done to ensure results are not changed should a major
compaction intercede. See <code>HBASE-15968</code> and linked issues for
discussion.</p>
</div>
<div class="paragraph">
<p>Running with this new configuration currently costs; we factor
the Cell MVCC on every compare so we burn more CPU. The slow
down will depend. In testing we&#8217;ve seen between 0% and 25%
degradation.</p>
</div>
<div class="paragraph">
<p>If replicating, it is advised that you run with the new
serial replication feature (See <code>HBASE-9465</code>; the serial
replication feature did NOT make it into <code>hbase-2.0.0</code> but
should arrive in a subsequent hbase-2.x release) as now
the order in which Mutations arrive is a factor.</p>
</div>
</div>
<div class="sect2">
<h3 id="_current_limitations"><a class="anchor" href="#_current_limitations"></a>29.4. Current Limitations</h3>
<div class="paragraph">
<p>The below limitations are addressed in hbase-2.0.0. See
the section above, <a href="#new.version.behavior">Optional New Version and Delete behavior in HBase-2.0.0</a>.</p>
</div>
<div class="sect3">
<h4 id="_deletes_mask_puts"><a class="anchor" href="#_deletes_mask_puts"></a>29.4.1. Deletes mask Puts</h4>
<div class="paragraph">
<p>Deletes mask puts, even puts that happened after the delete was entered.
See <a href="https://issues.apache.org/jira/browse/HBASE-2256">HBASE-2256</a>.
Remember that a delete writes a tombstone, which only disappears after then next major compaction has run.
Suppose you do a delete of everything &#8656; T.
After this you do a new put with a timestamp &#8656; T.
This put, even if it happened after the delete, will be masked by the delete tombstone.
Performing the put will not fail, but when you do a get you will notice the put did have no effect.
It will start working again after the major compaction has run.
These issues should not be a problem if you use always-increasing versions for new puts to a row.
But they can occur even if you do not care about time: just do delete and put immediately after each other, and there is some chance they happen within the same millisecond.</p>
</div>
</div>
<div class="sect3">
<h4 id="major.compactions.change.query.results"><a class="anchor" href="#major.compactions.change.query.results"></a>29.4.2. Major compactions change query results</h4>
<div class="paragraph">
<p><em>&#8230;&#8203;create three cell versions at t1, t2 and t3, with a maximum-versions
setting of 2. So when getting all versions, only the values at t2 and t3 will be
returned. But if you delete the version at t2 or t3, the one at t1 will appear again.
Obviously, once a major compaction has run, such behavior will not be the case
anymore&#8230;&#8203;</em> (See <em>Garbage Collection</em> in <a href="https://www.ngdata.com/bending-time-in-hbase/">Bending time in HBase</a>.)</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="dm.sort"><a class="anchor" href="#dm.sort"></a>30. Sort Order</h2>
<div class="sectionbody">
<div class="paragraph">
<p>All data model operations HBase return data in sorted order.
First by row, then by ColumnFamily, followed by column qualifier, and finally timestamp (sorted in reverse, so newest records are returned first).</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="dm.column.metadata"><a class="anchor" href="#dm.column.metadata"></a>31. Column Metadata</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There is no store of column metadata outside of the internal KeyValue instances for a ColumnFamily.
Thus, while HBase can support not only a wide number of columns per row, but a heterogeneous set of columns between rows as well, it is your responsibility to keep track of the column names.</p>
</div>
<div class="paragraph">
<p>The only way to get a complete set of columns that exist for a ColumnFamily is to process all the rows.
For more information about how HBase stores data internally, see <a href="#keyvalue">keyvalue</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="joins"><a class="anchor" href="#joins"></a>32. Joins</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Whether HBase supports joins is a common question on the dist-list, and there is a simple answer: it doesn&#8217;t, at not least in the way that RDBMS' support them (e.g., with equi-joins or outer-joins in SQL). As has been illustrated in this chapter, the read data model operations in HBase are Get and Scan.</p>
</div>
<div class="paragraph">
<p>However, that doesn&#8217;t mean that equivalent join functionality can&#8217;t be supported in your application, but you have to do it yourself.
The two primary strategies are either denormalizing the data upon writing to HBase, or to have lookup tables and do the join between HBase tables in your application or MapReduce code (and as RDBMS' demonstrate, there are several strategies for this depending on the size of the tables, e.g., nested loops vs.
hash-joins). So which is the best approach? It depends on what you are trying to do, and as such there isn&#8217;t a single answer that works for every use case.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_acid"><a class="anchor" href="#_acid"></a>33. ACID</h2>
<div class="sectionbody">
<div class="paragraph">
<p>See <a href="/acid-semantics.html">ACID Semantics</a>.
Lars Hofhansl has also written a note on <a href="http://hadoop-hbase.blogspot.com/2012/03/acid-in-hbase.html">ACID in HBase</a>.</p>
</div>
</div>
</div>
<h1 id="schema" class="sect0"><a class="anchor" href="#schema"></a>HBase and Schema Design</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>A good introduction on the strength and weaknesses modelling on the various non-rdbms datastores is
to be found in Ian Varley&#8217;s Master thesis,
<a href="http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf">No Relation: The Mixed Blessings of Non-Relational Databases</a>.
It is a little dated now but a good background read if you have a moment on how HBase schema modeling
differs from how it is done in an RDBMS. Also,
read <a href="#keyvalue">keyvalue</a> for how HBase stores data internally, and the section on <a href="#schema.casestudies">schema.casestudies</a>.</p>
</div>
<div class="paragraph">
<p>The documentation on the Cloud Bigtable website, <a href="https://cloud.google.com/bigtable/docs/schema-design">Designing Your Schema</a>,
is pertinent and nicely done and lessons learned there equally apply here in HBase land; just divide
any quoted values by ~10 to get what works for HBase: e.g. where it says individual values can be ~10MBs in size, HBase can do similar&#8201;&#8212;&#8201;perhaps best
to go smaller if you can&#8201;&#8212;&#8201;and where it says a maximum of 100 column families in Cloud Bigtable, think ~10 when
modeling on HBase.</p>
</div>
<div class="paragraph">
<p>See also Robert Yokota&#8217;s <a href="https://blogs.apache.org/hbase/entry/hbase-application-archetypes-redux">HBase Application Archetypes</a>
(an update on work done by other HBasers), for a helpful categorization of use cases that do well on top of the HBase model.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="schema.creation"><a class="anchor" href="#schema.creation"></a>34. Schema Creation</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase schemas can be created or updated using the <a href="#shell">The Apache HBase Shell</a> or by using <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html">Admin</a> in the Java API.</p>
</div>
<div class="paragraph">
<p>Tables must be disabled when making ColumnFamily modifications, for example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
Admin admin = <span class="keyword">new</span> Admin(conf);
TableName table = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);
admin.disableTable(table);
HColumnDescriptor cf1 = ...;
admin.addColumn(table, cf1); <span class="comment">// adding new ColumnFamily</span>
HColumnDescriptor cf2 = ...;
admin.modifyColumn(table, cf2); <span class="comment">// modifying existing ColumnFamily</span>
admin.enableTable(table);</code></pre>
</div>
</div>
<div class="paragraph">
<p>See <a href="#client_dependencies">client dependencies</a> for more information about configuring client connections.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
online schema changes are supported in the 0.92.x codebase, but the 0.90.x codebase requires the table to be disabled.
</td>
</tr>
</table>
</div>
<div class="sect2">
<h3 id="schema.updates"><a class="anchor" href="#schema.updates"></a>34.1. Schema Updates</h3>
<div class="paragraph">
<p>When changes are made to either Tables or ColumnFamilies (e.g. region size, block size), these changes take effect the next time there is a major compaction and the StoreFiles get re-written.</p>
</div>
<div class="paragraph">
<p>See <a href="#store">store</a> for more information on StoreFiles.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="table_schema_rules_of_thumb"><a class="anchor" href="#table_schema_rules_of_thumb"></a>35. Table Schema Rules Of Thumb</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are many different data sets, with different access patterns and service-level
expectations. Therefore, these rules of thumb are only an overview. Read the rest
of this chapter to get more details after you have gone through this list.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Aim to have regions sized between 10 and 50 GB.</p>
</li>
<li>
<p>Aim to have cells no larger than 10 MB, or 50 MB if you use <a href="#hbase_mob">mob</a>. Otherwise,
consider storing your cell data in HDFS and store a pointer to the data in HBase.</p>
</li>
<li>
<p>A typical schema has between 1 and 3 column families per table. HBase tables should
not be designed to mimic RDBMS tables.</p>
</li>
<li>
<p>Around 50-100 regions is a good number for a table with 1 or 2 column families.
Remember that a region is a contiguous segment of a column family.</p>
</li>
<li>
<p>Keep your column family names as short as possible. The column family names are
stored for every value (ignoring prefix encoding). They should not be self-documenting
and descriptive like in a typical RDBMS.</p>
</li>
<li>
<p>If you are storing time-based machine data or logging information, and the row key
is based on device ID or service ID plus time, you can end up with a pattern where
older data regions never have additional writes beyond a certain age. In this type
of situation, you end up with a small number of active regions and a large number
of older regions which have no new writes. For these situations, you can tolerate
a larger number of regions because your resource consumption is driven by the active
regions only.</p>
</li>
<li>
<p>If only one column family is busy with writes, only that column family accomulates
memory. Be aware of write patterns when allocating resources.</p>
</li>
</ul>
</div>
</div>
</div>
<h1 id="regionserver_sizing_rules_of_thumb" class="sect0"><a class="anchor" href="#regionserver_sizing_rules_of_thumb"></a>RegionServer Sizing Rules of Thumb</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>Lars Hofhansl wrote a great
<a href="http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html">blog post</a>
about RegionServer memory sizing. The upshot is that you probably need more memory
than you think you need. He goes into the impact of region size, memstore size, HDFS
replication factor, and other things to check.</p>
</div>
<div class="quoteblock">
<blockquote>
<div class="paragraph">
<p>Personally I would place the maximum disk space per machine that can be served
exclusively with HBase around 6T, unless you have a very read-heavy workload.
In that case the Java heap should be 32GB (20G regions, 128M memstores, the rest
defaults).</p>
</div>
</blockquote>
<div class="attribution">
&#8212; Lars Hofhansl<br>
<cite>http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html</cite>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="number.of.cfs"><a class="anchor" href="#number.of.cfs"></a>36. On the number of column families</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase currently does not do well with anything above two or three column families so keep the number of column families in your schema low.
Currently, flushing and compactions are done on a per Region basis so if one column family is carrying the bulk of the data bringing on flushes, the adjacent families will also be flushed even though the amount of data they carry is small.
When many column families exist the flushing and compaction interaction can make for a bunch of needless i/o (To be addressed by changing flushing and compaction to work on a per column family basis). For more information on compactions, see <a href="#compaction">Compaction</a>.</p>
</div>
<div class="paragraph">
<p>Try to make do with one column family if you can in your schemas.
Only introduce a second and third column family in the case where data access is usually column scoped; i.e.
you query one column family or the other but usually not both at the one time.</p>
</div>
<div class="sect2">
<h3 id="number.of.cfs.card"><a class="anchor" href="#number.of.cfs.card"></a>36.1. Cardinality of ColumnFamilies</h3>
<div class="paragraph">
<p>Where multiple ColumnFamilies exist in a single table, be aware of the cardinality (i.e., number of rows). If ColumnFamilyA has 1 million rows and ColumnFamilyB has 1 billion rows, ColumnFamilyA&#8217;s data will likely be spread across many, many regions (and RegionServers). This makes mass scans for ColumnFamilyA less efficient.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="rowkey.design"><a class="anchor" href="#rowkey.design"></a>37. Rowkey Design</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_hotspotting"><a class="anchor" href="#_hotspotting"></a>37.1. Hotspotting</h3>
<div class="paragraph">
<p>Rows in HBase are sorted lexicographically by row key.
This design optimizes for scans, allowing you to store related rows, or rows that will be read together, near each other.
However, poorly designed row keys are a common source of <em class="firstterm">hotspotting</em>.
Hotspotting occurs when a large amount of client traffic is directed at one node, or only a few nodes, of a cluster.
This traffic may represent reads, writes, or other operations.
The traffic overwhelms the single machine responsible for hosting that region, causing performance degradation and potentially leading to region unavailability.
This can also have adverse effects on other regions hosted by the same region server as that host is unable to service the requested load.
It is important to design data access patterns such that the cluster is fully and evenly utilized.</p>
</div>
<div class="paragraph">
<p>To prevent hotspotting on writes, design your row keys such that rows that truly do need to be in the same region are, but in the bigger picture, data is being written to multiple regions across the cluster, rather than one at a time.
Some common techniques for avoiding hotspotting are described below, along with some of their advantages and drawbacks.</p>
</div>
<div class="paragraph">
<div class="title">Salting</div>
<p>Salting in this sense has nothing to do with cryptography, but refers to adding random data to the start of a row key.
In this case, salting refers to adding a randomly-assigned prefix to the row key to cause it to sort differently than it otherwise would.
The number of possible prefixes correspond to the number of regions you want to spread the data across.
Salting can be helpful if you have a few "hot" row key patterns which come up over and over amongst other more evenly-distributed rows.
Consider the following example, which shows that salting can spread write load across multiple RegionServers, and illustrates some of the negative implications for reads.</p>
</div>
<div class="exampleblock">
<div class="title">Example 11. Salting Example</div>
<div class="content">
<div class="paragraph">
<p>Suppose you have the following list of row keys, and your table is split such that there is one region for each letter of the alphabet.
Prefix 'a' is one region, prefix 'b' is another.
In this table, all rows starting with 'f' are in the same region.
This example focuses on rows with keys like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>foo0001
foo0002
foo0003
foo0004</pre>
</div>
</div>
<div class="paragraph">
<p>Now, imagine that you would like to spread these across four different regions.
You decide to use four different salts: <code>a</code>, <code>b</code>, <code>c</code>, and <code>d</code>.
In this scenario, each of these letter prefixes will be on a different region.
After applying the salts, you have the following rowkeys instead.
Since you can now write to four separate regions, you theoretically have four times the throughput when writing that you would have if all the writes were going to the same region.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>a-foo0003
b-foo0001
c-foo0004
d-foo0002</pre>
</div>
</div>
<div class="paragraph">
<p>Then, if you add another row, it will randomly be assigned one of the four possible salt values and end up near one of the existing rows.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>a-foo0003
b-foo0001
c-foo0003
c-foo0004
d-foo0002</pre>
</div>
</div>
<div class="paragraph">
<p>Since this assignment will be random, you will need to do more work if you want to retrieve the rows in lexicographic order.
In this way, salting attempts to increase throughput on writes, but has a cost during reads.</p>
</div>
</div>
</div>
<div class="paragraph">
<div class="title">Hashing</div>
<p>Instead of a random assignment, you could use a one-way <em class="firstterm">hash</em> that would cause a given row to always be "salted" with the same prefix, in a way that would spread the load across the RegionServers, but allow for predictability during reads.
Using a deterministic hash allows the client to reconstruct the complete rowkey and use a Get operation to retrieve that row as normal.</p>
</div>
<div class="exampleblock">
<div class="title">Example 12. Hashing Example</div>
<div class="content">
Given the same situation in the salting example above, you could instead apply a one-way hash that would cause the row with key <code>foo0003</code> to always, and predictably, receive the <code>a</code> prefix.
Then, to retrieve that row, you would already know the key.
You could also optimize things so that certain pairs of keys were always in the same region, for instance.
</div>
</div>
<div class="paragraph">
<div class="title">Reversing the Key</div>
<p>A third common trick for preventing hotspotting is to reverse a fixed-width or numeric row key so that the part that changes the most often (the least significant digit) is first.
This effectively randomizes row keys, but sacrifices row ordering properties.</p>
</div>
<div class="paragraph">
<p>See <a href="https://communities.intel.com/community/itpeernetwork/datastack/blog/2013/11/10/discussion-on-designing-hbase-tables" class="bare">https://communities.intel.com/community/itpeernetwork/datastack/blog/2013/11/10/discussion-on-designing-hbase-tables</a>, and <a href="https://phoenix.apache.org/salted.html">article on Salted Tables</a> from the Phoenix project, and the discussion in the comments of <a href="https://issues.apache.org/jira/browse/HBASE-11682">HBASE-11682</a> for more information about avoiding hotspotting.</p>
</div>
</div>
<div class="sect2">
<h3 id="timeseries"><a class="anchor" href="#timeseries"></a>37.2. Monotonically Increasing Row Keys/Timeseries Data</h3>
<div class="paragraph">
<p>In the HBase chapter of Tom White&#8217;s book <a href="http://oreilly.com/catalog/9780596521981">Hadoop: The Definitive Guide</a> (O&#8217;Reilly) there is a an optimization note on watching out for a phenomenon where an import process walks in lock-step with all clients in concert pounding one of the table&#8217;s regions (and thus, a single node), then moving onto the next region, etc.
With monotonically increasing row-keys (i.e., using a timestamp), this will happen.
See this comic by IKai Lan on why monotonically increasing row keys are problematic in BigTable-like datastores: <a href="http://ikaisays.com/2011/01/25/app-engine-datastore-tip-monotonically-increasing-values-are-bad/">monotonically increasing values are bad</a>.
The pile-up on a single region brought on by monotonically increasing keys can be mitigated by randomizing the input records to not be in sorted order, but in general it&#8217;s best to avoid using a timestamp or a sequence (e.g. 1, 2, 3) as the row-key.</p>
</div>
<div class="paragraph">
<p>If you do need to upload time series data into HBase, you should study <a href="http://opentsdb.net/">OpenTSDB</a> as a successful example.
It has a page describing the <a href="http://opentsdb.net/schema.html">schema</a> it uses in HBase.
The key format in OpenTSDB is effectively [metric_type][event_timestamp], which would appear at first glance to contradict the previous advice about not using a timestamp as the key.
However, the difference is that the timestamp is not in the <em>lead</em> position of the key, and the design assumption is that there are dozens or hundreds (or more) of different metric types.
Thus, even with a continual stream of input data with a mix of metric types, the Puts are distributed across various points of regions in the table.</p>
</div>
<div class="paragraph">
<p>See <a href="#schema.casestudies">schema.casestudies</a> for some rowkey design examples.</p>
</div>
</div>
<div class="sect2">
<h3 id="keysize"><a class="anchor" href="#keysize"></a>37.3. Try to minimize row and column sizes</h3>
<div class="paragraph">
<p>In HBase, values are always freighted with their coordinates; as a cell value passes through the system, it&#8217;ll be accompanied by its row, column name, and timestamp - always.
If your rows and column names are large, especially compared to the size of the cell value, then you may run up against some interesting scenarios.
One such is the case described by Marc Limotte at the tail of <a href="https://issues.apache.org/jira/browse/HBASE-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&amp;focusedCommentId=13005272#comment-13005272">HBASE-3551</a> (recommended!). Therein, the indices that are kept on HBase storefiles (<a href="#hfile">StoreFile (HFile)</a>) to facilitate random access may end up occupying large chunks of the HBase allotted RAM because the cell value coordinates are large.
Mark in the above cited comment suggests upping the block size so entries in the store file index happen at a larger interval or modify the table schema so it makes for smaller rows and column names.
Compression will also make for larger indices.
See the thread <a href="http://search-hadoop.com/m/hemBv1LiN4Q1/a+question+storefileIndexSize&amp;subj=a+question+storefileIndexSize">a question storefileIndexSize</a> up on the user mailing list.</p>
</div>
<div class="paragraph">
<p>Most of the time small inefficiencies don&#8217;t matter all that much. Unfortunately, this is a case where they do.
Whatever patterns are selected for ColumnFamilies, attributes, and rowkeys they could be repeated several billion times in your data.</p>
</div>
<div class="paragraph">
<p>See <a href="#keyvalue">keyvalue</a> for more information on HBase stores data internally to see why this is important.</p>
</div>
<div class="sect3">
<h4 id="keysize.cf"><a class="anchor" href="#keysize.cf"></a>37.3.1. Column Families</h4>
<div class="paragraph">
<p>Try to keep the ColumnFamily names as small as possible, preferably one character (e.g. "d" for data/default).</p>
</div>
<div class="paragraph">
<p>See <a href="#keyvalue">KeyValue</a> for more information on HBase stores data internally to see why this is important.</p>
</div>
</div>
<div class="sect3">
<h4 id="keysize.attributes"><a class="anchor" href="#keysize.attributes"></a>37.3.2. Attributes</h4>
<div class="paragraph">
<p>Although verbose attribute names (e.g., "myVeryImportantAttribute") are easier to read, prefer shorter attribute names (e.g., "via") to store in HBase.</p>
</div>
<div class="paragraph">
<p>See <a href="#keyvalue">keyvalue</a> for more information on HBase stores data internally to see why this is important.</p>
</div>
</div>
<div class="sect3">
<h4 id="keysize.row"><a class="anchor" href="#keysize.row"></a>37.3.3. Rowkey Length</h4>
<div class="paragraph">
<p>Keep them as short as is reasonable such that they can still be useful for required data access (e.g. Get vs.
Scan). A short key that is useless for data access is not better than a longer key with better get/scan properties.
Expect tradeoffs when designing rowkeys.</p>
</div>
</div>
<div class="sect3">
<h4 id="keysize.patterns"><a class="anchor" href="#keysize.patterns"></a>37.3.4. Byte Patterns</h4>
<div class="paragraph">
<p>A long is 8 bytes.
You can store an unsigned number up to 18,446,744,073,709,551,615 in those eight bytes.
If you stored this number as a String&#8201;&#8212;&#8201;presuming a byte per character&#8201;&#8212;&#8201;you need nearly 3x the bytes.</p>
</div>
<div class="paragraph">
<p>Not convinced? Below is some sample code that you can run on your own.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">// long</span>
<span class="comment">//</span>
<span class="type">long</span> l = <span class="integer">1234567890L</span>;
<span class="type">byte</span><span class="type">[]</span> lb = Bytes.toBytes(l);
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">long bytes length: </span><span class="delimiter">&quot;</span></span> + lb.length); <span class="comment">// returns 8</span>
<span class="predefined-type">String</span> s = <span class="predefined-type">String</span>.valueOf(l);
<span class="type">byte</span><span class="type">[]</span> sb = Bytes.toBytes(s);
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">long as string length: </span><span class="delimiter">&quot;</span></span> + sb.length); <span class="comment">// returns 10</span>
<span class="comment">// hash</span>
<span class="comment">//</span>
<span class="predefined-type">MessageDigest</span> md = <span class="predefined-type">MessageDigest</span>.getInstance(<span class="string"><span class="delimiter">&quot;</span><span class="content">MD5</span><span class="delimiter">&quot;</span></span>);
<span class="type">byte</span><span class="type">[]</span> digest = md.digest(Bytes.toBytes(s));
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">md5 digest bytes length: </span><span class="delimiter">&quot;</span></span> + digest.length); <span class="comment">// returns 16</span>
<span class="predefined-type">String</span> sDigest = <span class="keyword">new</span> <span class="predefined-type">String</span>(digest);
<span class="type">byte</span><span class="type">[]</span> sbDigest = Bytes.toBytes(sDigest);
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">md5 digest as string length: </span><span class="delimiter">&quot;</span></span> + sbDigest.length); <span class="comment">// returns 26</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Unfortunately, using a binary representation of a type will make your data harder to read outside of your code.
For example, this is what you will see in the shell when you increment a value:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; incr <span class="string"><span class="delimiter">'</span><span class="content">t</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">f:q</span><span class="delimiter">'</span></span>, <span class="integer">1</span>
COUNTER VALUE = <span class="integer">1</span>
hbase(main):<span class="octal">002</span>:<span class="integer">0</span>&gt; get <span class="string"><span class="delimiter">'</span><span class="content">t</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r</span><span class="delimiter">'</span></span>
COLUMN CELL
f:q timestamp=<span class="integer">1369163040570</span>, value=<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x00<span class="error">\</span>x01
<span class="integer">1</span> row(s) in <span class="float">0.0310</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>The shell makes a best effort to print a string, and it this case it decided to just print the hex.
The same will happen to your row keys inside the region names.
It can be okay if you know what&#8217;s being stored, but it might also be unreadable if arbitrary data can be put in the same cells.
This is the main trade-off.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="reverse.timestamp"><a class="anchor" href="#reverse.timestamp"></a>37.4. Reverse Timestamps</h3>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Reverse Scan API</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-4811">HBASE-4811</a> implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
This feature is available in HBase 0.98 and later.
See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed-boolean-">Scan.setReversed()</a> for more information.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>A common problem in database processing is quickly finding the most recent version of a value.
A technique using reverse timestamps as a part of the key can help greatly with a special case of this problem.
Also found in the HBase chapter of Tom White&#8217;s book Hadoop: The Definitive Guide (O&#8217;Reilly), the technique involves appending (<code>Long.MAX_VALUE - timestamp</code>) to the end of any key, e.g. [key][reverse_timestamp].</p>
</div>
<div class="paragraph">
<p>The most recent value for [key] in a table can be found by performing a Scan for [key] and obtaining the first record.
Since HBase keys are in sorted order, this key sorts before any older row-keys for [key] and thus is first.</p>
</div>
<div class="paragraph">
<p>This technique would be used instead of using <a href="#schema.versions">Number of Versions</a> where the intent is to hold onto all versions "forever" (or a very long time) and at the same time quickly obtain access to any other version by using the same Scan technique.</p>
</div>
</div>
<div class="sect2">
<h3 id="rowkey.scope"><a class="anchor" href="#rowkey.scope"></a>37.5. Rowkeys and ColumnFamilies</h3>
<div class="paragraph">
<p>Rowkeys are scoped to ColumnFamilies.
Thus, the same rowkey could exist in each ColumnFamily that exists in a table without collision.</p>
</div>
</div>
<div class="sect2">
<h3 id="changing.rowkeys"><a class="anchor" href="#changing.rowkeys"></a>37.6. Immutability of Rowkeys</h3>
<div class="paragraph">
<p>Rowkeys cannot be changed.
The only way they can be "changed" in a table is if the row is deleted and then re-inserted.
This is a fairly common question on the HBase dist-list so it pays to get the rowkeys right the first time (and/or before you&#8217;ve inserted a lot of data).</p>
</div>
</div>
<div class="sect2">
<h3 id="rowkey.regionsplits"><a class="anchor" href="#rowkey.regionsplits"></a>37.7. Relationship Between RowKeys and Region Splits</h3>
<div class="paragraph">
<p>If you pre-split your table, it is <em>critical</em> to understand how your rowkey will be distributed across the region boundaries.
As an example of why this is important, consider the example of using displayable hex characters as the lead position of the key (e.g., "0000000000000000" to "ffffffffffffffff"). Running those key ranges through <code>Bytes.split</code> (which is the split strategy used when creating regions in <code>Admin.createTable(byte[] startKey, byte[] endKey, numRegions)</code> for 10 regions will generate the following splits&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre>48 48 48 48 48 48 48 48 48 48 48 48 48 48 48 48 // 0
54 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 // 6
61 -67 -67 -67 -67 -67 -67 -67 -67 -67 -67 -67 -67 -67 -67 -68 // =
68 -124 -124 -124 -124 -124 -124 -124 -124 -124 -124 -124 -124 -124 -124 -126 // D
75 75 75 75 75 75 75 75 75 75 75 75 75 75 75 72 // K
82 18 18 18 18 18 18 18 18 18 18 18 18 18 18 14 // R
88 -40 -40 -40 -40 -40 -40 -40 -40 -40 -40 -40 -40 -40 -40 -44 // X
95 -97 -97 -97 -97 -97 -97 -97 -97 -97 -97 -97 -97 -97 -97 -102 // _
102 102 102 102 102 102 102 102 102 102 102 102 102 102 102 102 // f</pre>
</div>
</div>
<div class="paragraph">
<p>(note: the lead byte is listed to the right as a comment.) Given that the first split is a '0' and the last split is an 'f', everything is great, right? Not so fast.</p>
</div>
<div class="paragraph">
<p>The problem is that all the data is going to pile up in the first 2 regions and the last region thus creating a "lumpy" (and possibly "hot") region problem.
To understand why, refer to an <a href="http://www.asciitable.com">ASCII Table</a>.
'0' is byte 48, and 'f' is byte 102, but there is a huge gap in byte values (bytes 58 to 96) that will <em>never appear in this keyspace</em> because the only values are [0-9] and [a-f]. Thus, the middle regions will never be used.
To make pre-splitting work with this example keyspace, a custom definition of splits (i.e., and not relying on the built-in split method) is required.</p>
</div>
<div class="paragraph">
<p>Lesson #1: Pre-splitting tables is generally a best practice, but you need to pre-split them in such a way that all the regions are accessible in the keyspace.
While this example demonstrated the problem with a hex-key keyspace, the same problem can happen with <em>any</em> keyspace.
Know your data.</p>
</div>
<div class="paragraph">
<p>Lesson #2: While generally not advisable, using hex-keys (and more generally, displayable data) can still work with pre-split tables as long as all the created regions are accessible in the keyspace.</p>
</div>
<div class="paragraph">
<p>To conclude this example, the following is an example of how appropriate splits can be pre-created for hex-keys:.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">boolean</span> createTable(Admin admin, HTableDescriptor table, <span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> splits)
<span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">try</span> {
admin.createTable( table, splits );
<span class="keyword">return</span> <span class="predefined-constant">true</span>;
} <span class="keyword">catch</span> (TableExistsException e) {
logger.info(<span class="string"><span class="delimiter">&quot;</span><span class="content">table </span><span class="delimiter">&quot;</span></span> + table.getNameAsString() + <span class="string"><span class="delimiter">&quot;</span><span class="content"> already exists</span><span class="delimiter">&quot;</span></span>);
<span class="comment">// the table already exists...</span>
<span class="keyword">return</span> <span class="predefined-constant">false</span>;
}
}
<span class="directive">public</span> <span class="directive">static</span> <span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> getHexSplits(<span class="predefined-type">String</span> startKey, <span class="predefined-type">String</span> endKey, <span class="type">int</span> numRegions) {
<span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> splits = <span class="keyword">new</span> <span class="type">byte</span>[numRegions-<span class="integer">1</span>]<span class="type">[]</span>;
<span class="predefined-type">BigInteger</span> lowestKey = <span class="keyword">new</span> <span class="predefined-type">BigInteger</span>(startKey, <span class="integer">16</span>);
<span class="predefined-type">BigInteger</span> highestKey = <span class="keyword">new</span> <span class="predefined-type">BigInteger</span>(endKey, <span class="integer">16</span>);
<span class="predefined-type">BigInteger</span> range = highestKey.subtract(lowestKey);
<span class="predefined-type">BigInteger</span> regionIncrement = range.divide(<span class="predefined-type">BigInteger</span>.valueOf(numRegions));
lowestKey = lowestKey.add(regionIncrement);
<span class="keyword">for</span>(<span class="type">int</span> i=<span class="integer">0</span>; i &lt; numRegions-<span class="integer">1</span>;i++) {
<span class="predefined-type">BigInteger</span> key = lowestKey.add(regionIncrement.multiply(<span class="predefined-type">BigInteger</span>.valueOf(i)));
<span class="type">byte</span><span class="type">[]</span> b = <span class="predefined-type">String</span>.format(<span class="string"><span class="delimiter">&quot;</span><span class="content">%016x</span><span class="delimiter">&quot;</span></span>, key).getBytes();
splits[i] = b;
}
<span class="keyword">return</span> splits;
}</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="schema.versions"><a class="anchor" href="#schema.versions"></a>38. Number of Versions</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="schema.versions.max"><a class="anchor" href="#schema.versions.max"></a>38.1. Maximum Number of Versions</h3>
<div class="paragraph">
<p>The maximum number of row versions to store is configured per column family via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a>.
The default for max versions is 1.
This is an important parameter because as described in <a href="#datamodel">Data Model</a> section HBase does <em>not</em> overwrite row values, but rather stores different values per row by time (and qualifier). Excess versions are removed during major compactions.
The number of max versions may need to be increased or decreased depending on application needs.</p>
</div>
<div class="paragraph">
<p>It is not recommended setting the number of max versions to an exceedingly high level (e.g., hundreds or more) unless those old values are very dear to you because this will greatly increase StoreFile size.</p>
</div>
</div>
<div class="sect2">
<h3 id="schema.minversions"><a class="anchor" href="#schema.minversions"></a>38.2. Minimum Number of Versions</h3>
<div class="paragraph">
<p>Like maximum number of row versions, the minimum number of row versions to keep is configured per column family via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a>.
The default for min versions is 0, which means the feature is disabled.
The minimum number of row versions parameter is used together with the time-to-live parameter and can be combined with the number of row versions parameter to allow configurations such as "keep the last T minutes worth of data, at most N versions, <em>but keep at least M versions around</em>" (where M is the value for minimum number of row versions, M&lt;N). This parameter should only be set when time-to-live is enabled for a column family and must be less than the number of row versions.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="supported.datatypes"><a class="anchor" href="#supported.datatypes"></a>39. Supported Datatypes</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase supports a "bytes-in/bytes-out" interface via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html">Put</a> and <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Result.html">Result</a>, so anything that can be converted to an array of bytes can be stored as a value.
Input could be strings, numbers, complex objects, or even images as long as they can rendered as bytes.</p>
</div>
<div class="paragraph">
<p>There are practical limits to the size of values (e.g., storing 10-50MB objects in HBase would probably be too much to ask); search the mailing list for conversations on this topic.
All rows in HBase conform to the <a href="#datamodel">Data Model</a>, and that includes versioning.
Take that into consideration when making your design, as well as block size for the ColumnFamily.</p>
</div>
<div class="sect2">
<h3 id="_counters"><a class="anchor" href="#_counters"></a>39.1. Counters</h3>
<div class="paragraph">
<p>One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#increment%28org.apache.hadoop.hbase.client.Increment%29">Increment</a> in <code>Table</code>.</p>
</div>
<div class="paragraph">
<p>Synchronization on counters are done on the RegionServer, not in the client.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="schema.joins"><a class="anchor" href="#schema.joins"></a>40. Joins</h2>
<div class="sectionbody">
<div class="paragraph">
<p>If you have multiple tables, don&#8217;t forget to factor in the potential for <a href="#joins">Joins</a> into the schema design.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ttl"><a class="anchor" href="#ttl"></a>41. Time To Live (TTL)</h2>
<div class="sectionbody">
<div class="paragraph">
<p>ColumnFamilies can set a TTL length in seconds, and HBase will automatically delete rows once the expiration time is reached.
This applies to <em>all</em> versions of a row - even the current one.
The TTL time encoded in the HBase for the row is specified in UTC.</p>
</div>
<div class="paragraph">
<p>Store files which contains only expired rows are deleted on minor compaction.
Setting <code>hbase.store.delete.expired.storefile</code> to <code>false</code> disables this feature.
Setting minimum number of versions to other than 0 also disables this.</p>
</div>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a> for more information.</p>
</div>
<div class="paragraph">
<p>Recent versions of HBase also support setting time to live on a per cell basis.
See <a href="https://issues.apache.org/jira/browse/HBASE-10560">HBASE-10560</a> for more information.
Cell TTLs are submitted as an attribute on mutation requests (Appends, Increments, Puts, etc.) using Mutation#setTTL.
If the TTL attribute is set, it will be applied to all cells updated on the server by the operation.
There are two notable differences between cell TTL handling and ColumnFamily TTLs:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Cell TTLs are expressed in units of milliseconds instead of seconds.</p>
</li>
<li>
<p>A cell TTLs cannot extend the effective lifetime of a cell beyond a ColumnFamily level TTL setting.</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="cf.keep.deleted"><a class="anchor" href="#cf.keep.deleted"></a>42. Keeping Deleted Cells</h2>
<div class="sectionbody">
<div class="paragraph">
<p>By default, delete markers extend back to the beginning of time.
Therefore, <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</a> or <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> operations will not see a deleted cell (row or column), even when the Get or Scan operation indicates a time range before the delete marker was placed.</p>
</div>
<div class="paragraph">
<p>ColumnFamilies can optionally keep deleted cells.
In this case, deleted cells can still be retrieved, as long as these operations specify a time range that ends before the timestamp of any delete that would affect the cells.
This allows for point-in-time queries even in the presence of deletes.</p>
</div>
<div class="paragraph">
<p>Deleted cells are still subject to TTL and there will never be more than "maximum number of versions" deleted cells.
A new "raw" scan options returns all deleted rows and the delete markers.</p>
</div>
<div class="listingblock">
<div class="title">Change the Value of <code>KEEP_DELETED_CELLS</code> Using HBase Shell</div>
<div class="content">
<pre>hbase&gt; hbase&gt; alter ‘t1′, NAME =&gt; ‘f1′, KEEP_DELETED_CELLS =&gt; true</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 13. Change the Value of <code>KEEP_DELETED_CELLS</code> Using the API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
HColumnDescriptor.setKeepDeletedCells(<span class="predefined-constant">true</span>);
...</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>Let us illustrate the basic effect of setting the <code>KEEP_DELETED_CELLS</code> attribute on a table.</p>
</div>
<div class="paragraph">
<p>First, without:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">create <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {NAME=&gt;<span class="string"><span class="delimiter">'</span><span class="content">e</span><span class="delimiter">'</span></span>, VERSIONS=&gt;<span class="integer">2147483647</span>}
put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">10</span>
put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">12</span>
put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">14</span>
delete <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="integer">11</span>
hbase(main):<span class="octal">017</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">11</span>, type=DeleteColumn
r1 column=e:c1, timestamp=<span class="integer">10</span>, value=value
<span class="integer">1</span> row(s) in <span class="float">0.0120</span> seconds
hbase(main):<span class="integer">018</span>:<span class="integer">0</span>&gt; flush <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>
<span class="integer">0</span> row(s) in <span class="float">0.0350</span> seconds
hbase(main):<span class="integer">019</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">11</span>, type=DeleteColumn
<span class="integer">1</span> row(s) in <span class="float">0.0120</span> seconds
hbase(main):<span class="octal">020</span>:<span class="integer">0</span>&gt; major_compact <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>
<span class="integer">0</span> row(s) in <span class="float">0.0260</span> seconds
hbase(main):<span class="octal">021</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
<span class="integer">1</span> row(s) in <span class="float">0.0120</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>Notice how delete cells are let go.</p>
</div>
<div class="paragraph">
<p>Now let&#8217;s run the same test only with <code>KEEP_DELETED_CELLS</code> set on the table (you can do table or per-column-family):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">005</span>:<span class="integer">0</span>&gt; create <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {NAME=&gt;<span class="string"><span class="delimiter">'</span><span class="content">e</span><span class="delimiter">'</span></span>, VERSIONS=&gt;<span class="integer">2147483647</span>, KEEP_DELETED_CELLS =&gt; <span class="predefined-constant">true</span>}
<span class="integer">0</span> row(s) in <span class="float">0.2160</span> seconds
=&gt; Hbase::Table - test
hbase(main):<span class="octal">006</span>:<span class="integer">0</span>&gt; put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">10</span>
<span class="integer">0</span> row(s) in <span class="float">0.1070</span> seconds
hbase(main):<span class="octal">007</span>:<span class="integer">0</span>&gt; put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">12</span>
<span class="integer">0</span> row(s) in <span class="float">0.0140</span> seconds
hbase(main):<span class="integer">008</span>:<span class="integer">0</span>&gt; put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, <span class="integer">14</span>
<span class="integer">0</span> row(s) in <span class="float">0.0160</span> seconds
hbase(main):<span class="integer">009</span>:<span class="integer">0</span>&gt; delete <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">r1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">e:c1</span><span class="delimiter">'</span></span>, <span class="integer">11</span>
<span class="integer">0</span> row(s) in <span class="float">0.0290</span> seconds
hbase(main):<span class="octal">010</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">11</span>, type=DeleteColumn
r1 column=e:c1, timestamp=<span class="integer">10</span>, value=value
<span class="integer">1</span> row(s) in <span class="float">0.0550</span> seconds
hbase(main):<span class="octal">011</span>:<span class="integer">0</span>&gt; flush <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>
<span class="integer">0</span> row(s) in <span class="float">0.2780</span> seconds
hbase(main):<span class="octal">012</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">11</span>, type=DeleteColumn
r1 column=e:c1, timestamp=<span class="integer">10</span>, value=value
<span class="integer">1</span> row(s) in <span class="float">0.0620</span> seconds
hbase(main):<span class="octal">013</span>:<span class="integer">0</span>&gt; major_compact <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>
<span class="integer">0</span> row(s) in <span class="float">0.0530</span> seconds
hbase(main):<span class="octal">014</span>:<span class="integer">0</span>&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {RAW=&gt;<span class="predefined-constant">true</span>, VERSIONS=&gt;<span class="integer">1000</span>}
ROW COLUMN+CELL
r1 column=e:c1, timestamp=<span class="integer">14</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">12</span>, value=value
r1 column=e:c1, timestamp=<span class="integer">11</span>, type=DeleteColumn
r1 column=e:c1, timestamp=<span class="integer">10</span>, value=value
<span class="integer">1</span> row(s) in <span class="float">0.0650</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>KEEP_DELETED_CELLS is to avoid removing Cells from HBase when the <em>only</em> reason to remove them is the delete marker.
So with KEEP_DELETED_CELLS enabled deleted cells would get removed if either you write more versions than the configured max, or you have a TTL and Cells are in excess of the configured timeout, etc.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="secondary.indexes"><a class="anchor" href="#secondary.indexes"></a>43. Secondary Indexes and Alternate Query Paths</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This section could also be titled "what if my table rowkey looks like <em>this</em> but I also want to query my table like <em>that</em>." A common example on the dist-list is where a row-key is of the format "user-timestamp" but there are reporting requirements on activity across users for certain time ranges.
Thus, selecting by user is easy because it is in the lead position of the key, but time is not.</p>
</div>
<div class="paragraph">
<p>There is no single answer on the best way to handle this because it depends on&#8230;&#8203;</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Number of users</p>
</li>
<li>
<p>Data size and data arrival rate</p>
</li>
<li>
<p>Flexibility of reporting requirements (e.g., completely ad-hoc date selection vs. pre-configured ranges)</p>
</li>
<li>
<p>Desired execution speed of query (e.g., 90 seconds may be reasonable to some for an ad-hoc report, whereas it may be too long for others)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>and solutions are also influenced by the size of the cluster and how much processing power you have to throw at the solution.
Common techniques are in sub-sections below.
This is a comprehensive, but not exhaustive, list of approaches.</p>
</div>
<div class="paragraph">
<p>It should not be a surprise that secondary indexes require additional cluster space and processing.
This is precisely what happens in an RDBMS because the act of creating an alternate index requires both space and processing cycles to update.
RDBMS products are more advanced in this regard to handle alternative index management out of the box.
However, HBase scales better at larger data volumes, so this is a feature trade-off.</p>
</div>
<div class="paragraph">
<p>Pay attention to <a href="#performance">Apache HBase Performance Tuning</a> when implementing any of these approaches.</p>
</div>
<div class="paragraph">
<p>Additionally, see the David Butler response in this dist-list thread <a href="http://search-hadoop.com/m/nvbiBp2TDP/Stargate%252Bhbase&amp;subj=Stargate+hbase">HBase, mail # user - Stargate+hbase</a></p>
</div>
<div class="sect2">
<h3 id="secondary.indexes.filter"><a class="anchor" href="#secondary.indexes.filter"></a>43.1. Filter Query</h3>
<div class="paragraph">
<p>Depending on the case, it may be appropriate to use <a href="#client.filter">Client Request Filters</a>.
In this case, no secondary index is created.
However, don&#8217;t try a full-scan on a large table like this from an application (i.e., single-threaded client).</p>
</div>
</div>
<div class="sect2">
<h3 id="secondary.indexes.periodic"><a class="anchor" href="#secondary.indexes.periodic"></a>43.2. Periodic-Update Secondary Index</h3>
<div class="paragraph">
<p>A secondary index could be created in another table which is periodically updated via a MapReduce job.
The job could be executed intra-day, but depending on load-strategy it could still potentially be out of sync with the main data table.</p>
</div>
<div class="paragraph">
<p>See <a href="#mapreduce.example.readwrite">mapreduce.example.readwrite</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="secondary.indexes.dualwrite"><a class="anchor" href="#secondary.indexes.dualwrite"></a>43.3. Dual-Write Secondary Index</h3>
<div class="paragraph">
<p>Another strategy is to build the secondary index while publishing data to the cluster (e.g., write to data table, write to index table). If this is approach is taken after a data table already exists, then bootstrapping will be needed for the secondary index with a MapReduce job (see <a href="#secondary.indexes.periodic">secondary.indexes.periodic</a>).</p>
</div>
</div>
<div class="sect2">
<h3 id="secondary.indexes.summary"><a class="anchor" href="#secondary.indexes.summary"></a>43.4. Summary Tables</h3>
<div class="paragraph">
<p>Where time-ranges are very wide (e.g., year-long report) and where the data is voluminous, summary tables are a common approach.
These would be generated with MapReduce jobs into another table.</p>
</div>
<div class="paragraph">
<p>See <a href="#mapreduce.example.summary">mapreduce.example.summary</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="secondary.indexes.coproc"><a class="anchor" href="#secondary.indexes.coproc"></a>43.5. Coprocessor Secondary Index</h3>
<div class="paragraph">
<p>Coprocessors act like RDBMS triggers. These were added in 0.92.
For more information, see <a href="#cp">coprocessors</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_constraints"><a class="anchor" href="#_constraints"></a>44. Constraints</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase currently supports 'constraints' in traditional (SQL) database parlance.
The advised usage for Constraints is in enforcing business rules for attributes
in the table (e.g. make sure values are in the range 1-10). Constraints could
also be used to enforce referential integrity, but this is strongly discouraged
as it will dramatically decrease the write throughput of the tables where integrity
checking is enabled. Extensive documentation on using Constraints can be found at
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/constraint/Constraint.html">Constraint</a>
since version 0.94.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="schema.casestudies"><a class="anchor" href="#schema.casestudies"></a>45. Schema Design Case Studies</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The following will describe some typical data ingestion use-cases with HBase, and how the rowkey design and construction can be approached.
Note: this is just an illustration of potential approaches, not an exhaustive list.
Know your data, and know your processing requirements.</p>
</div>
<div class="paragraph">
<p>It is highly recommended that you read the rest of the <a href="#schema">HBase and Schema Design</a> first, before reading these case studies.</p>
</div>
<div class="paragraph">
<p>The following case studies are described:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Log Data / Timeseries Data</p>
</li>
<li>
<p>Log Data / Timeseries on Steroids</p>
</li>
<li>
<p>Customer/Order</p>
</li>
<li>
<p>Tall/Wide/Middle Schema Design</p>
</li>
<li>
<p>List Data</p>
</li>
</ul>
</div>
<div class="sect2">
<h3 id="schema.casestudies.log_timeseries"><a class="anchor" href="#schema.casestudies.log_timeseries"></a>45.1. Case Study - Log Data and Timeseries Data</h3>
<div class="paragraph">
<p>Assume that the following data elements are being collected.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Hostname</p>
</li>
<li>
<p>Timestamp</p>
</li>
<li>
<p>Log event</p>
</li>
<li>
<p>Value/message</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>We can store them in an HBase table called LOG_DATA, but what will the rowkey be? From these attributes the rowkey will be some combination of hostname, timestamp, and log-event - but what specifically?</p>
</div>
<div class="sect3">
<h4 id="schema.casestudies.log_timeseries.tslead"><a class="anchor" href="#schema.casestudies.log_timeseries.tslead"></a>45.1.1. Timestamp In The Rowkey Lead Position</h4>
<div class="paragraph">
<p>The rowkey <code>[timestamp][hostname][log-event]</code> suffers from the monotonically increasing rowkey problem described in <a href="#timeseries">Monotonically Increasing Row Keys/Timeseries Data</a>.</p>
</div>
<div class="paragraph">
<p>There is another pattern frequently mentioned in the dist-lists about "bucketing" timestamps, by performing a mod operation on the timestamp.
If time-oriented scans are important, this could be a useful approach.
Attention must be paid to the number of buckets, because this will require the same number of scans to return results.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="type">long</span> bucket = timestamp % numBuckets;</code></pre>
</div>
</div>
<div class="paragraph">
<p>to construct:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">[bucket][timestamp][hostname][log-event]</code></pre>
</div>
</div>
<div class="paragraph">
<p>As stated above, to select data for a particular timerange, a Scan will need to be performed for each bucket.
100 buckets, for example, will provide a wide distribution in the keyspace but it will require 100 Scans to obtain data for a single timestamp, so there are trade-offs.</p>
</div>
</div>
<div class="sect3">
<h4 id="schema.casestudies.log_timeseries.hostlead"><a class="anchor" href="#schema.casestudies.log_timeseries.hostlead"></a>45.1.2. Host In The Rowkey Lead Position</h4>
<div class="paragraph">
<p>The rowkey <code>[hostname][log-event][timestamp]</code> is a candidate if there is a large-ish number of hosts to spread the writes and reads across the keyspace.
This approach would be useful if scanning by hostname was a priority.</p>
</div>
</div>
<div class="sect3">
<h4 id="schema.casestudies.log_timeseries.revts"><a class="anchor" href="#schema.casestudies.log_timeseries.revts"></a>45.1.3. Timestamp, or Reverse Timestamp?</h4>
<div class="paragraph">
<p>If the most important access path is to pull most recent events, then storing the timestamps as reverse-timestamps (e.g., <code>timestamp = Long.MAX_VALUE – timestamp</code>) will create the property of being able to do a Scan on <code>[hostname][log-event]</code> to obtain the most recently captured events.</p>
</div>
<div class="paragraph">
<p>Neither approach is wrong, it just depends on what is most appropriate for the situation.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Reverse Scan API</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-4811">HBASE-4811</a> implements an API to scan a table or a range within a table in reverse, reducing the need to optimize your schema for forward or reverse scanning.
This feature is available in HBase 0.98 and later.
See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setReversed-boolean-">Scan.setReversed()</a> for more information.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="schema.casestudies.log_timeseries.varkeys"><a class="anchor" href="#schema.casestudies.log_timeseries.varkeys"></a>45.1.4. Variable Length or Fixed Length Rowkeys?</h4>
<div class="paragraph">
<p>It is critical to remember that rowkeys are stamped on every column in HBase.
If the hostname is <code>a</code> and the event type is <code>e1</code> then the resulting rowkey would be quite small.
However, what if the ingested hostname is <code>myserver1.mycompany.com</code> and the event type is <code>com.package1.subpackage2.subsubpackage3.ImportantService</code>?</p>
</div>
<div class="paragraph">
<p>It might make sense to use some substitution in the rowkey.
There are at least two approaches: hashed and numeric.
In the Hostname In The Rowkey Lead Position example, it might look like this:</p>
</div>
<div class="paragraph">
<p>Composite Rowkey With Hashes:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[MD5 hash of hostname] = 16 bytes</p>
</li>
<li>
<p>[MD5 hash of event-type] = 16 bytes</p>
</li>
<li>
<p>[timestamp] = 8 bytes</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Composite Rowkey With Numeric Substitution:</p>
</div>
<div class="paragraph">
<p>For this approach another lookup table would be needed in addition to LOG_DATA, called LOG_TYPES.
The rowkey of LOG_TYPES would be:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[type]</code> (e.g., byte indicating hostname vs. event-type)</p>
</li>
<li>
<p><code>[bytes]</code> variable length bytes for raw hostname or event-type.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>A column for this rowkey could be a long with an assigned number, which could be obtained
by using an <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#incrementColumnValue-byte:A-byte:A-byte:A-long-">HBase counter</a></p>
</div>
<div class="paragraph">
<p>So the resulting composite rowkey would be:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[substituted long for hostname] = 8 bytes</p>
</li>
<li>
<p>[substituted long for event type] = 8 bytes</p>
</li>
<li>
<p>[timestamp] = 8 bytes</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>In either the Hash or Numeric substitution approach, the raw values for hostname and event-type can be stored as columns.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="schema.casestudies.log_steroids"><a class="anchor" href="#schema.casestudies.log_steroids"></a>45.2. Case Study - Log Data and Timeseries Data on Steroids</h3>
<div class="paragraph">
<p>This effectively is the OpenTSDB approach.
What OpenTSDB does is re-write data and pack rows into columns for certain time-periods.
For a detailed explanation, see: <a href="http://opentsdb.net/schema.html" class="bare">http://opentsdb.net/schema.html</a>, and
<a href="https://www.slideshare.net/cloudera/4-opentsdb-hbasecon">Lessons Learned from OpenTSDB</a>
from HBaseCon2012.</p>
</div>
<div class="paragraph">
<p>But this is how the general concept works: data is ingested, for example, in this manner&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre>[hostname][log-event][timestamp1]
[hostname][log-event][timestamp2]
[hostname][log-event][timestamp3]</pre>
</div>
</div>
<div class="paragraph">
<p>with separate rowkeys for each detailed event, but is re-written like this&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre>[hostname][log-event][timerange]</pre>
</div>
</div>
<div class="paragraph">
<p>and each of the above events are converted into columns stored with a time-offset relative to the beginning timerange (e.g., every 5 minutes). This is obviously a very advanced processing technique, but HBase makes this possible.</p>
</div>
</div>
<div class="sect2">
<h3 id="schema.casestudies.custorder"><a class="anchor" href="#schema.casestudies.custorder"></a>45.3. Case Study - Customer/Order</h3>
<div class="paragraph">
<p>Assume that HBase is used to store customer and order information.
There are two core record-types being ingested: a Customer record type, and Order record type.</p>
</div>
<div class="paragraph">
<p>The Customer record type would include all the things that you&#8217;d typically expect:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Customer number</p>
</li>
<li>
<p>Customer name</p>
</li>
<li>
<p>Address (e.g., city, state, zip)</p>
</li>
<li>
<p>Phone numbers, etc.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The Order record type would include things like:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Customer number</p>
</li>
<li>
<p>Order number</p>
</li>
<li>
<p>Sales date</p>
</li>
<li>
<p>A series of nested objects for shipping locations and line-items (see <a href="#schema.casestudies.custorder.obj">Order Object Design</a> for details)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Assuming that the combination of customer number and sales order uniquely identify an order, these two attributes will compose the rowkey, and specifically a composite key such as:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>[customer number][order number]</pre>
</div>
</div>
<div class="paragraph">
<p>for an ORDER table.
However, there are more design decisions to make: are the <em>raw</em> values the best choices for rowkeys?</p>
</div>
<div class="paragraph">
<p>The same design questions in the Log Data use-case confront us here.
What is the keyspace of the customer number, and what is the format (e.g., numeric? alphanumeric?) As it is advantageous to use fixed-length keys in HBase, as well as keys that can support a reasonable spread in the keyspace, similar options appear:</p>
</div>
<div class="paragraph">
<p>Composite Rowkey With Hashes:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[MD5 of customer number] = 16 bytes</p>
</li>
<li>
<p>[MD5 of order number] = 16 bytes</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Composite Numeric/Hash Combo Rowkey:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[substituted long for customer number] = 8 bytes</p>
</li>
<li>
<p>[MD5 of order number] = 16 bytes</p>
</li>
</ul>
</div>
<div class="sect3">
<h4 id="schema.casestudies.custorder.tables"><a class="anchor" href="#schema.casestudies.custorder.tables"></a>45.3.1. Single Table? Multiple Tables?</h4>
<div class="paragraph">
<p>A traditional design approach would have separate tables for CUSTOMER and SALES.
Another option is to pack multiple record types into a single table (e.g., CUSTOMER++).</p>
</div>
<div class="paragraph">
<p>Customer Record Type Rowkey:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[customer-id]</p>
</li>
<li>
<p>[type] = type indicating `1' for customer record type</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Order Record Type Rowkey:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>[customer-id]</p>
</li>
<li>
<p>[type] = type indicating `2' for order record type</p>
</li>
<li>
<p>[order]</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The advantage of this particular CUSTOMER++ approach is that organizes many different record-types by customer-id (e.g., a single scan could get you everything about that customer). The disadvantage is that it&#8217;s not as easy to scan for a particular record-type.</p>
</div>
</div>
<div class="sect3">
<h4 id="schema.casestudies.custorder.obj"><a class="anchor" href="#schema.casestudies.custorder.obj"></a>45.3.2. Order Object Design</h4>
<div class="paragraph">
<p>Now we need to address how to model the Order object.
Assume that the class structure is as follows:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Order</dt>
<dd>
<p>(an Order can have multiple ShippingLocations</p>
</dd>
<dt class="hdlist1">LineItem</dt>
<dd>
<p>(a ShippingLocation can have multiple LineItems</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>there are multiple options on storing this data.</p>
</div>
<div class="sect4">
<h5 id="schema.casestudies.custorder.obj.norm"><a class="anchor" href="#schema.casestudies.custorder.obj.norm"></a>Completely Normalized</h5>
<div class="paragraph">
<p>With this approach, there would be separate tables for ORDER, SHIPPING_LOCATION, and LINE_ITEM.</p>
</div>
<div class="paragraph">
<p>The ORDER table&#8217;s rowkey was described above: <a href="#schema.casestudies.custorder">schema.casestudies.custorder</a></p>
</div>
<div class="paragraph">
<p>The SHIPPING_LOCATION&#8217;s composite rowkey would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[shipping location number]</code> (e.g., 1st location, 2nd, etc.)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The LINE_ITEM table&#8217;s composite rowkey would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[shipping location number]</code> (e.g., 1st location, 2nd, etc.)</p>
</li>
<li>
<p><code>[line item number]</code> (e.g., 1st lineitem, 2nd, etc.)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Such a normalized model is likely to be the approach with an RDBMS, but that&#8217;s not your only option with HBase.
The cons of such an approach is that to retrieve information about any Order, you will need:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Get on the ORDER table for the Order</p>
</li>
<li>
<p>Scan on the SHIPPING_LOCATION table for that order to get the ShippingLocation instances</p>
</li>
<li>
<p>Scan on the LINE_ITEM for each ShippingLocation</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>granted, this is what an RDBMS would do under the covers anyway, but since there are no joins in HBase you&#8217;re just more aware of this fact.</p>
</div>
</div>
<div class="sect4">
<h5 id="schema.casestudies.custorder.obj.rectype"><a class="anchor" href="#schema.casestudies.custorder.obj.rectype"></a>Single Table With Record Types</h5>
<div class="paragraph">
<p>With this approach, there would exist a single table ORDER that would contain</p>
</div>
<div class="paragraph">
<p>The Order rowkey was described above: <a href="#schema.casestudies.custorder">schema.casestudies.custorder</a></p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[ORDER record type]</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The ShippingLocation composite rowkey would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[SHIPPING record type]</code></p>
</li>
<li>
<p><code>[shipping location number]</code> (e.g., 1st location, 2nd, etc.)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The LineItem composite rowkey would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[LINE record type]</code></p>
</li>
<li>
<p><code>[shipping location number]</code> (e.g., 1st location, 2nd, etc.)</p>
</li>
<li>
<p><code>[line item number]</code> (e.g., 1st lineitem, 2nd, etc.)</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="schema.casestudies.custorder.obj.denorm"><a class="anchor" href="#schema.casestudies.custorder.obj.denorm"></a>Denormalized</h5>
<div class="paragraph">
<p>A variant of the Single Table With Record Types approach is to denormalize and flatten some of the object hierarchy, such as collapsing the ShippingLocation attributes onto each LineItem instance.</p>
</div>
<div class="paragraph">
<p>The LineItem composite rowkey would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>[order-rowkey]</code></p>
</li>
<li>
<p><code>[LINE record type]</code></p>
</li>
<li>
<p><code>[line item number]</code> (e.g., 1st lineitem, 2nd, etc., care must be taken that there are unique across the entire order)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>and the LineItem columns would be something like this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>itemNumber</p>
</li>
<li>
<p>quantity</p>
</li>
<li>
<p>price</p>
</li>
<li>
<p>shipToLine1 (denormalized from ShippingLocation)</p>
</li>
<li>
<p>shipToLine2 (denormalized from ShippingLocation)</p>
</li>
<li>
<p>shipToCity (denormalized from ShippingLocation)</p>
</li>
<li>
<p>shipToState (denormalized from ShippingLocation)</p>
</li>
<li>
<p>shipToZip (denormalized from ShippingLocation)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The pros of this approach include a less complex object hierarchy, but one of the cons is that updating gets more complicated in case any of this information changes.</p>
</div>
</div>
<div class="sect4">
<h5 id="schema.casestudies.custorder.obj.singleobj"><a class="anchor" href="#schema.casestudies.custorder.obj.singleobj"></a>Object BLOB</h5>
<div class="paragraph">
<p>With this approach, the entire Order object graph is treated, in one way or another, as a BLOB.
For example, the ORDER table&#8217;s rowkey was described above: <a href="#schema.casestudies.custorder">schema.casestudies.custorder</a>, and a single column called "order" would contain an object that could be deserialized that contained a container Order, ShippingLocations, and LineItems.</p>
</div>
<div class="paragraph">
<p>There are many options here: JSON, XML, Java Serialization, Avro, Hadoop Writables, etc.
All of them are variants of the same approach: encode the object graph to a byte-array.
Care should be taken with this approach to ensure backward compatibility in case the object model changes such that older persisted structures can still be read back out of HBase.</p>
</div>
<div class="paragraph">
<p>Pros are being able to manage complex object graphs with minimal I/O (e.g., a single HBase Get per Order in this example), but the cons include the aforementioned warning about backward compatibility of serialization, language dependencies of serialization (e.g., Java Serialization only works with Java clients), the fact that you have to deserialize the entire object to get any piece of information inside the BLOB, and the difficulty in getting frameworks like Hive to work with custom objects like this.</p>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="schema.smackdown"><a class="anchor" href="#schema.smackdown"></a>45.4. Case Study - "Tall/Wide/Middle" Schema Design Smackdown</h3>
<div class="paragraph">
<p>This section will describe additional schema design questions that appear on the dist-list, specifically about tall and wide tables.
These are general guidelines and not laws - each application must consider its own needs.</p>
</div>
<div class="sect3">
<h4 id="schema.smackdown.rowsversions"><a class="anchor" href="#schema.smackdown.rowsversions"></a>45.4.1. Rows vs. Versions</h4>
<div class="paragraph">
<p>A common question is whether one should prefer rows or HBase&#8217;s built-in-versioning.
The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 1 max versions). The rows-approach would require storing a timestamp in some portion of the rowkey so that they would not overwrite with each successive update.</p>
</div>
<div class="paragraph">
<p>Preference: Rows (generally speaking).</p>
</div>
</div>
<div class="sect3">
<h4 id="schema.smackdown.rowscols"><a class="anchor" href="#schema.smackdown.rowscols"></a>45.4.2. Rows vs. Columns</h4>
<div class="paragraph">
<p>Another common question is whether one should prefer rows or columns.
The context is typically in extreme cases of wide tables, such as having 1 row with 1 million attributes, or 1 million rows with 1 columns apiece.</p>
</div>
<div class="paragraph">
<p>Preference: Rows (generally speaking). To be clear, this guideline is in the context is in extremely wide cases, not in the standard use-case where one needs to store a few dozen or hundred columns.
But there is also a middle path between these two options, and that is "Rows as Columns."</p>
</div>
</div>
<div class="sect3">
<h4 id="schema.smackdown.rowsascols"><a class="anchor" href="#schema.smackdown.rowsascols"></a>45.4.3. Rows as Columns</h4>
<div class="paragraph">
<p>The middle path between Rows vs.
Columns is packing data that would be a separate row into columns, for certain rows.
OpenTSDB is the best example of this case where a single row represents a defined time-range, and then discrete events are treated as columns.
This approach is often more complex, and may require the additional complexity of re-writing your data, but has the advantage of being I/O efficient.
For an overview of this approach, see <a href="#schema.casestudies.log_steroids">schema.casestudies.log-steroids</a>.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="casestudies.schema.listdata"><a class="anchor" href="#casestudies.schema.listdata"></a>45.5. Case Study - List Data</h3>
<div class="paragraph">
<p>The following is an exchange from the user dist-list regarding a fairly common question: how to handle per-user list data in Apache HBase.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>QUESTION <strong>*</strong></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>We&#8217;re looking at how to store a large amount of (per-user) list data in HBase, and we were trying to figure out what kind of access pattern made the most sense.
One option is store the majority of the data in a key, so we could have something like:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">&lt;FixedWidthUserName&gt;&lt;FixedWidthValueId1&gt;:<span class="string"><span class="delimiter">&quot;</span><span class="delimiter">&quot;</span></span> (no value)
&lt;FixedWidthUserName&gt;&lt;FixedWidthValueId2&gt;:<span class="string"><span class="delimiter">&quot;</span><span class="delimiter">&quot;</span></span> (no value)
&lt;FixedWidthUserName&gt;&lt;FixedWidthValueId3&gt;:<span class="string"><span class="delimiter">&quot;</span><span class="delimiter">&quot;</span></span> (no value)</code></pre>
</div>
</div>
<div class="paragraph">
<p>The other option we had was to do this entirely using:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;FixedWidthUserName&gt;</span><span class="tag">&lt;FixedWidthPageNum0&gt;</span>:<span class="tag">&lt;FixedWidthLength&gt;</span><span class="tag">&lt;FixedIdNextPageNum&gt;</span><span class="tag">&lt;ValueId1&gt;</span><span class="tag">&lt;ValueId2&gt;</span><span class="tag">&lt;ValueId3&gt;</span>...
<span class="tag">&lt;FixedWidthUserName&gt;</span><span class="tag">&lt;FixedWidthPageNum1&gt;</span>:<span class="tag">&lt;FixedWidthLength&gt;</span><span class="tag">&lt;FixedIdNextPageNum&gt;</span><span class="tag">&lt;ValueId1&gt;</span><span class="tag">&lt;ValueId2&gt;</span><span class="tag">&lt;ValueId3&gt;</span>...</code></pre>
</div>
</div>
<div class="paragraph">
<p>where each row would contain multiple values.
So in one case reading the first thirty values would be:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">scan { STARTROW =&gt; <span class="string"><span class="delimiter">'</span><span class="content">FixedWidthUsername</span><span class="delimiter">'</span></span> LIMIT =&gt; <span class="integer">30</span>}</code></pre>
</div>
</div>
<div class="paragraph">
<p>And in the second case it would be</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">get <span class="string"><span class="delimiter">'</span><span class="content">FixedWidthUserName</span><span class="content">\x00</span><span class="content">\x00</span><span class="content">\x00</span><span class="content">\x00</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The general usage pattern would be to read only the first 30 values of these lists, with infrequent access reading deeper into the lists.
Some users would have &#8656; 30 total values in these lists, and some users would have millions (i.e.
power-law distribution)</p>
</div>
<div class="paragraph">
<p>The single-value format seems like it would take up more space on HBase, but would offer some improved retrieval / pagination flexibility.
Would there be any significant performance advantages to be able to paginate via gets vs paginating with scans?</p>
</div>
<div class="paragraph">
<p>My initial understanding was that doing a scan should be faster if our paging size is unknown (and caching is set appropriately), but that gets should be faster if we&#8217;ll always need the same page size.
I&#8217;ve ended up hearing different people tell me opposite things about performance.
I assume the page sizes would be relatively consistent, so for most use cases we could guarantee that we only wanted one page of data in the fixed-page-length case.
I would also assume that we would have infrequent updates, but may have inserts into the middle of these lists (meaning we&#8217;d need to update all subsequent rows).</p>
</div>
<div class="paragraph">
<p>Thanks for help / suggestions / follow-up questions.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>ANSWER <strong>*</strong></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>If I understand you correctly, you&#8217;re ultimately trying to store triples in the form "user, valueid, value", right? E.g., something like:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">user123, firstname, Paul</span><span class="delimiter">&quot;</span></span>,
<span class="string"><span class="delimiter">&quot;</span><span class="content">user234, lastname, Smith</span><span class="delimiter">&quot;</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>(But the usernames are fixed width, and the valueids are fixed width).</p>
</div>
<div class="paragraph">
<p>And, your access pattern is along the lines of: "for user X, list the next 30 values, starting with valueid Y". Is that right? And these values should be returned sorted by valueid?</p>
</div>
<div class="paragraph">
<p>The tl;dr version is that you should probably go with one row per user+value, and not build a complicated intra-row pagination scheme on your own unless you&#8217;re really sure it is needed.</p>
</div>
<div class="paragraph">
<p>Your two options mirror a common question people have when designing HBase schemas: should I go "tall" or "wide"? Your first schema is "tall": each row represents one value for one user, and so there are many rows in the table for each user; the row key is user + valueid, and there would be (presumably) a single column qualifier that means "the value". This is great if you want to scan over rows in sorted order by row key (thus my question above, about whether these ids are sorted correctly). You can start a scan at any user+valueid, read the next 30, and be done.
What you&#8217;re giving up is the ability to have transactional guarantees around all the rows for one user, but it doesn&#8217;t sound like you need that.
Doing it this way is generally recommended (see here <a href="https://hbase.apache.org/book.html#schema.smackdown" class="bare">https://hbase.apache.org/book.html#schema.smackdown</a>).</p>
</div>
<div class="paragraph">
<p>Your second option is "wide": you store a bunch of values in one row, using different qualifiers (where the qualifier is the valueid). The simple way to do that would be to just store ALL values for one user in a single row.
I&#8217;m guessing you jumped to the "paginated" version because you&#8217;re assuming that storing millions of columns in a single row would be bad for performance, which may or may not be true; as long as you&#8217;re not trying to do too much in a single request, or do things like scanning over and returning all of the cells in the row, it shouldn&#8217;t be fundamentally worse.
The client has methods that allow you to get specific slices of columns.</p>
</div>
<div class="paragraph">
<p>Note that neither case fundamentally uses more disk space than the other; you&#8217;re just "shifting" part of the identifying information for a value either to the left (into the row key, in option one) or to the right (into the column qualifiers in option 2). Under the covers, every key/value still stores the whole row key, and column family name.
(If this is a bit confusing, take an hour and watch Lars George&#8217;s excellent video about understanding HBase schema design: <a href="http://www.youtube.com/watch?v=_HLoH_PgrLk" class="bare">http://www.youtube.com/watch?v=_HLoH_PgrLk</a>).</p>
</div>
<div class="paragraph">
<p>A manually paginated version has lots more complexities, as you note, like having to keep track of how many things are in each page, re-shuffling if new values are inserted, etc.
That seems significantly more complex.
It might have some slight speed advantages (or disadvantages!) at extremely high throughput, and the only way to really know that would be to try it out.
If you don&#8217;t have time to build it both ways and compare, my advice would be to start with the simplest option (one row per user+value). Start simple and iterate! :)</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="schema.ops"><a class="anchor" href="#schema.ops"></a>46. Operational and Performance Configuration Options</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_tune_hbase_server_rpc_handling"><a class="anchor" href="#_tune_hbase_server_rpc_handling"></a>46.1. Tune HBase Server RPC Handling</h3>
<div class="ulist">
<ul>
<li>
<p>Set <code>hbase.regionserver.handler.count</code> (in <code>hbase-site.xml</code>) to cores x spindles for concurrency.</p>
</li>
<li>
<p>Optionally, split the call queues into separate read and write queues for differentiated service. The parameter <code>hbase.ipc.server.callqueue.handler.factor</code> specifies the number of call queues:</p>
<div class="ulist">
<ul>
<li>
<p><code>0</code> means a single shared queue</p>
</li>
<li>
<p><code>1</code> means one queue for each handler.</p>
</li>
<li>
<p>A value between <code>0</code> and <code>1</code> allocates the number of queues proportionally to the number of handlers. For instance, a value of <code>.5</code> shares one queue between each two handlers.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Use <code>hbase.ipc.server.callqueue.read.ratio</code> (<code>hbase.ipc.server.callqueue.read.share</code> in 0.98) to split the call queues into read and write queues:</p>
<div class="ulist">
<ul>
<li>
<p><code>0.5</code> means there will be the same number of read and write queues</p>
</li>
<li>
<p><code>&lt; 0.5</code> for more read than write</p>
</li>
<li>
<p><code>&gt; 0.5</code> for more write than read</p>
</li>
</ul>
</div>
</li>
<li>
<p>Set <code>hbase.ipc.server.callqueue.scan.ratio</code> (HBase 1.0+) to split read call queues into small-read and long-read queues:</p>
<div class="ulist">
<ul>
<li>
<p>0.5 means that there will be the same number of short-read and long-read queues</p>
</li>
<li>
<p><code>&lt; 0.5</code> for more short-read</p>
</li>
<li>
<p><code>&gt; 0.5</code> for more long-read</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_disable_nagle_for_rpc"><a class="anchor" href="#_disable_nagle_for_rpc"></a>46.2. Disable Nagle for RPC</h3>
<div class="paragraph">
<p>Disable Nagle’s algorithm. Delayed ACKs can add up to ~200ms to RPC round trip time. Set the following parameters:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>In Hadoop’s <code>core-site.xml</code>:</p>
<div class="ulist">
<ul>
<li>
<p><code>ipc.server.tcpnodelay = true</code></p>
</li>
<li>
<p><code>ipc.client.tcpnodelay = true</code></p>
</li>
</ul>
</div>
</li>
<li>
<p>In HBase’s <code>hbase-site.xml</code>:</p>
<div class="ulist">
<ul>
<li>
<p><code>hbase.ipc.client.tcpnodelay = true</code></p>
</li>
<li>
<p><code>hbase.ipc.server.tcpnodelay = true</code></p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_limit_server_failure_impact"><a class="anchor" href="#_limit_server_failure_impact"></a>46.3. Limit Server Failure Impact</h3>
<div class="paragraph">
<p>Detect regionserver failure as fast as reasonable. Set the following parameters:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>In <code>hbase-site.xml</code>, set <code>zookeeper.session.timeout</code> to 30 seconds or less to bound failure detection (20-30 seconds is a good start).</p>
<div class="ulist">
<ul>
<li>
<p>Notice: the <code>sessionTimeout</code> of zookeeper is limited between 2 times and 20 times the <code>tickTime</code>(the basic time unit in milliseconds used by ZooKeeper.the default value is 2000ms.It is used to do heartbeats and the minimum session timeout will be twice the tickTime).</p>
</li>
</ul>
</div>
</li>
<li>
<p>Detect and avoid unhealthy or failed HDFS DataNodes: in <code>hdfs-site.xml</code> and <code>hbase-site.xml</code>, set the following parameters:</p>
<div class="ulist">
<ul>
<li>
<p><code>dfs.namenode.avoid.read.stale.datanode = true</code></p>
</li>
<li>
<p><code>dfs.namenode.avoid.write.stale.datanode = true</code></p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="shortcircuit.reads"><a class="anchor" href="#shortcircuit.reads"></a>46.4. Optimize on the Server Side for Low Latency</h3>
<div class="paragraph">
<p>Skip the network for local blocks when the RegionServer goes to read from HDFS by exploiting HDFS&#8217;s
<a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html">Short-Circuit Local Reads</a> facility.
Note how setup must be done both at the datanode and on the dfsclient ends of the conneciton&#8201;&#8212;&#8201;i.e. at the RegionServer
and how both ends need to have loaded the hadoop native <code>.so</code> library.
After configuring your hadoop setting <em>dfs.client.read.shortcircuit</em> to <em>true</em> and configuring
the <em>dfs.domain.socket.path</em> path for the datanode and dfsclient to share and restarting, next configure
the regionserver/dfsclient side.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>In <code>hbase-site.xml</code>, set the following parameters:</p>
<div class="ulist">
<ul>
<li>
<p><code>dfs.client.read.shortcircuit = true</code></p>
</li>
<li>
<p><code>dfs.client.read.shortcircuit.skip.checksum = true</code> so we don&#8217;t double checksum (HBase does its own checksumming to save on i/os. See <a href="#hbase.regionserver.checksum.verify.performance"><code>hbase.regionserver.checksum.verify</code></a> for more on this.</p>
</li>
<li>
<p><code>dfs.domain.socket.path</code> to match what was set for the datanodes.</p>
</li>
<li>
<p><code>dfs.client.read.shortcircuit.buffer.size = 131072</code> Important to avoid OOME&#8201;&#8212;&#8201;hbase has a default it uses if unset, see <code>hbase.dfs.client.read.shortcircuit.buffer.size</code>; its default is 131072.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Ensure data locality. In <code>hbase-site.xml</code>, set <code>hbase.hstore.min.locality.to.skip.major.compact = 0.7</code> (Meaning that 0.7 &lt;= n &lt;= 1)</p>
</li>
<li>
<p>Make sure DataNodes have enough handlers for block transfers. In <code>hdfs-site.xml</code>, set the following parameters:</p>
<div class="ulist">
<ul>
<li>
<p><code>dfs.datanode.max.xcievers &gt;= 8192</code></p>
</li>
<li>
<p><code>dfs.datanode.handler.count =</code> number of spindles</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
<div class="paragraph">
<p>Check the RegionServer logs after restart. You should only see complaint if misconfiguration.
Otherwise, shortcircuit read operates quietly in background. It does not provide metrics so
no optics on how effective it is but read latencies should show a marked improvement, especially if
good data locality, lots of random reads, and dataset is larger than available cache.</p>
</div>
<div class="paragraph">
<p>Other advanced configurations that you might play with, especially if shortcircuit functionality
is complaining in the logs, include <code>dfs.client.read.shortcircuit.streams.cache.size</code> and
<code>dfs.client.socketcache.capacity</code>. Documentation is sparse on these options. You&#8217;ll have to
read source code.</p>
</div>
<div class="paragraph">
<p>For more on short-circuit reads, see Colin&#8217;s old blog on rollout,
<a href="http://blog.cloudera.com/blog/2013/08/how-improved-short-circuit-local-reads-bring-better-performance-and-security-to-hadoop/">How Improved Short-Circuit Local Reads Bring Better Performance and Security to Hadoop</a>.
The <a href="https://issues.apache.org/jira/browse/HDFS-347">HDFS-347</a> issue also makes for an
interesting read showing the HDFS community at its best (caveat a few comments).</p>
</div>
</div>
<div class="sect2">
<h3 id="_jvm_tuning"><a class="anchor" href="#_jvm_tuning"></a>46.5. JVM Tuning</h3>
<div class="sect3">
<h4 id="_tune_jvm_gc_for_low_collection_latencies"><a class="anchor" href="#_tune_jvm_gc_for_low_collection_latencies"></a>46.5.1. Tune JVM GC for low collection latencies</h4>
<div class="ulist">
<ul>
<li>
<p>Use the CMS collector: <code>-XX:+UseConcMarkSweepGC</code></p>
</li>
<li>
<p>Keep eden space as small as possible to minimize average collection time. Example:</p>
<div class="literalblock">
<div class="content">
<pre>-XX:CMSInitiatingOccupancyFraction=70</pre>
</div>
</div>
</li>
<li>
<p>Optimize for low collection latency rather than throughput: <code>-Xmn512m</code></p>
</li>
<li>
<p>Collect eden in parallel: <code>-XX:+UseParNewGC</code></p>
</li>
<li>
<p>Avoid collection under pressure: <code>-XX:+UseCMSInitiatingOccupancyOnly</code></p>
</li>
<li>
<p>Limit per request scanner result sizing so everything fits into survivor space but doesn’t tenure. In <code>hbase-site.xml</code>, set <code>hbase.client.scanner.max.result.size</code> to 1/8th of eden space (with -<code>Xmn512m</code> this is ~51MB )</p>
</li>
<li>
<p>Set <code>max.result.size</code> x <code>handler.count</code> less than survivor space</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_os_level_tuning"><a class="anchor" href="#_os_level_tuning"></a>46.5.2. OS-Level Tuning</h4>
<div class="ulist">
<ul>
<li>
<p>Turn transparent huge pages (THP) off:</p>
<div class="literalblock">
<div class="content">
<pre>echo never &gt; /sys/kernel/mm/transparent_hugepage/enabled
echo never &gt; /sys/kernel/mm/transparent_hugepage/defrag</pre>
</div>
</div>
</li>
<li>
<p>Set <code>vm.swappiness = 0</code></p>
</li>
<li>
<p>Set <code>vm.min_free_kbytes</code> to at least 1GB (8GB on larger memory systems)</p>
</li>
<li>
<p>Disable NUMA zone reclaim with <code>vm.zone_reclaim_mode = 0</code></p>
</li>
</ul>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_special_cases"><a class="anchor" href="#_special_cases"></a>47. Special Cases</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_for_applications_where_failing_quickly_is_better_than_waiting"><a class="anchor" href="#_for_applications_where_failing_quickly_is_better_than_waiting"></a>47.1. For applications where failing quickly is better than waiting</h3>
<div class="ulist">
<ul>
<li>
<p>In <code>hbase-site.xml</code> on the client side, set the following parameters:</p>
<div class="ulist">
<ul>
<li>
<p>Set <code>hbase.client.pause = 1000</code></p>
</li>
<li>
<p>Set <code>hbase.client.retries.number = 3</code></p>
</li>
<li>
<p>If you want to ride over splits and region moves, increase <code>hbase.client.retries.number</code> substantially (&gt;= 20)</p>
</li>
<li>
<p>Set the RecoverableZookeeper retry count: <code>zookeeper.recovery.retry = 1</code> (no retry)</p>
</li>
</ul>
</div>
</li>
<li>
<p>In <code>hbase-site.xml</code> on the server side, set the Zookeeper session timeout for detecting server failures: <code>zookeeper.session.timeout</code> &#8656; 30 seconds (20-30 is good).</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_for_applications_that_can_tolerate_slightly_out_of_date_information"><a class="anchor" href="#_for_applications_that_can_tolerate_slightly_out_of_date_information"></a>47.2. For applications that can tolerate slightly out of date information</h3>
<div class="paragraph">
<p><strong>HBase timeline consistency (HBASE-10070) </strong>
With read replicas enabled, read-only copies of regions (replicas) are distributed over the cluster. One RegionServer services the default or primary replica, which is the only replica that can service writes. Other RegionServers serve the secondary replicas, follow the primary RegionServer, and only see committed updates. The secondary replicas are read-only, but can serve reads immediately while the primary is failing over, cutting read availability blips from seconds to milliseconds. Phoenix supports timeline consistency as of 4.4.0
Tips:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Deploy HBase 1.0.0 or later.</p>
</li>
<li>
<p>Enable timeline consistent replicas on the server side.</p>
</li>
<li>
<p>Use one of the following methods to set timeline consistency:</p>
<div class="ulist">
<ul>
<li>
<p>Use <code>ALTER SESSION SET CONSISTENCY = 'TIMELINE’</code></p>
</li>
<li>
<p>Set the connection property <code>Consistency</code> to <code>timeline</code> in the JDBC connect string</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_more_information"><a class="anchor" href="#_more_information"></a>47.3. More Information</h3>
<div class="paragraph">
<p>See the Performance section <a href="#perf.schema">perf.schema</a> for more information about operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.</p>
</div>
</div>
</div>
</div>
<h1 id="mapreduce" class="sect0"><a class="anchor" href="#mapreduce"></a>HBase and MapReduce</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>Apache MapReduce is a software framework used to analyze large amounts of data. It is provided by <a href="https://hadoop.apache.org/">Apache Hadoop</a>.
MapReduce itself is out of the scope of this document.
A good place to get started with MapReduce is <a href="https://hadoop.apache.org/docs/r2.6.0/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html" class="bare">https://hadoop.apache.org/docs/r2.6.0/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html</a>.
MapReduce version 2 (MR2)is now part of <a href="https://hadoop.apache.org/docs/r2.3.0/hadoop-yarn/hadoop-yarn-site/">YARN</a>.</p>
</div>
<div class="paragraph">
<p>This chapter discusses specific configuration steps you need to take to use MapReduce on data within HBase.
In addition, it discusses other interactions and issues between HBase and MapReduce
jobs. Finally, it discusses <a href="#cascading">Cascading</a>, an
<a href="http://www.cascading.org/">alternative API</a> for MapReduce.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title"><code>mapred</code> and <code>mapreduce</code></div>
<div class="paragraph">
<p>There are two mapreduce packages in HBase as in MapReduce itself: <em>org.apache.hadoop.hbase.mapred</em> and <em>org.apache.hadoop.hbase.mapreduce</em>.
The former does old-style API and the latter the new mode.
The latter has more facility though you can usually find an equivalent in the older package.
Pick the package that goes with your MapReduce deploy.
When in doubt or starting over, pick <em>org.apache.hadoop.hbase.mapreduce</em>.
In the notes below, we refer to <em>o.a.h.h.mapreduce</em> but replace with
<em>o.a.h.h.mapred</em> if that is what you are using.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.mapreduce.classpath"><a class="anchor" href="#hbase.mapreduce.classpath"></a>48. HBase, MapReduce, and the CLASSPATH</h2>
<div class="sectionbody">
<div class="paragraph">
<p>By default, MapReduce jobs deployed to a MapReduce cluster do not have access to
either the HBase configuration under <code>$HBASE_CONF_DIR</code> or the HBase classes.</p>
</div>
<div class="paragraph">
<p>To give the MapReduce jobs the access they need, you could add <em>hbase-site.xml_to _$HADOOP_HOME/conf</em> and add HBase jars to the <em>$HADOOP_HOME/lib</em> directory.
You would then need to copy these changes across your cluster. Or you could edit <em>$HADOOP_HOME/conf/hadoop-env.sh</em> and add hbase dependencies to the <code>HADOOP_CLASSPATH</code> variable.
Neither of these approaches is recommended because it will pollute your Hadoop install with HBase references.
It also requires you restart the Hadoop cluster before Hadoop can use the HBase data.</p>
</div>
<div class="paragraph">
<p>The recommended approach is to let HBase add its dependency jars and use <code>HADOOP_CLASSPATH</code> or <code>-libjars</code>.</p>
</div>
<div class="paragraph">
<p>Since HBase <code>0.90.x</code>, HBase adds its dependency JARs to the job configuration itself.
The dependencies only need to be available on the local <code>CLASSPATH</code> and from here they&#8217;ll be picked
up and bundled into the fat job jar deployed to the MapReduce cluster. A basic trick just passes
the full hbase classpath&#8201;&#8212;&#8201;all hbase and dependent jars as well as configurations&#8201;&#8212;&#8201;to the mapreduce
job runner letting hbase utility pick out from the full-on classpath what it needs adding them to the
MapReduce job configuration (See the source at <code>TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job)</code> for how this is done).</p>
</div>
<div class="paragraph">
<p>The following example runs the bundled HBase <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html">RowCounter</a> MapReduce job against a table named <code>usertable</code>.
It sets into <code>HADOOP_CLASSPATH</code> the jars hbase needs to run in an MapReduce context (including configuration files such as hbase-site.xml).
Be sure to use the correct version of the HBase JAR for your system; replace the VERSION string in the below command line w/ the version of
your local hbase install. The backticks (<code>`</code> symbols) cause the shell to execute the sub-commands, setting the output of <code>hbase classpath</code> into <code>HADOOP_CLASSPATH</code>.
This example assumes you use a BASH-compatible shell.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` \
${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/lib/hbase-mapreduce-VERSION.jar \
org.apache.hadoop.hbase.mapreduce.RowCounter usertable</code></pre>
</div>
</div>
<div class="paragraph">
<p>The above command will launch a row counting mapreduce job against the hbase cluster that is pointed to by your local configuration on a cluster that the hadoop configs are pointing to.</p>
</div>
<div class="paragraph">
<p>The main for the <code>hbase-mapreduce.jar</code> is a Driver that lists a few basic mapreduce tasks that ship with hbase.
For example, presuming your install is hbase <code>2.0.0-SNAPSHOT</code>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` \
${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/lib/hbase-mapreduce-2.0.0-SNAPSHOT.jar
An example program must be given as the first argument.
Valid program names are:
CellCounter: Count cells in HBase table.
WALPlayer: Replay WAL files.
completebulkload: Complete a bulk data load.
copytable: Export a table from local cluster to peer cluster.
export: Write table data to HDFS.
exportsnapshot: Export the specific snapshot to a given FileSystem.
import: Import data written by Export.
importtsv: Import data in TSV format.
rowcounter: Count rows in HBase table.
verifyrep: Compare the data from tables in two different clusters. WARNING: It doesn't work for incrementColumnValues'd cells since the timestamp is changed after being appended to the log.</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can use the above listed shortnames for mapreduce jobs as in the below re-run of the row counter job (again, presuming your install is hbase <code>2.0.0-SNAPSHOT</code>):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` \
${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/lib/hbase-mapreduce-2.0.0-SNAPSHOT.jar \
rowcounter usertable</code></pre>
</div>
</div>
<div class="paragraph">
<p>You might find the more selective <code>hbase mapredcp</code> tool output of interest; it lists the minimum set of jars needed
to run a basic mapreduce job against an hbase install. It does not include configuration. You&#8217;ll probably need to add
these if you want your MapReduce job to find the target cluster. You&#8217;ll probably have to also add pointers to extra jars
once you start to do anything of substance. Just specify the extras by passing the system propery <code>-Dtmpjars</code> when
you run <code>hbase mapredcp</code>.</p>
</div>
<div class="paragraph">
<p>For jobs that do not package their dependencies or call <code>TableMapReduceUtil#addDependencyJars</code>, the following command structure is necessary:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase mapredcp`:${HBASE_HOME}/conf hadoop jar MyApp.jar MyJobMainClass -libjars $(${HBASE_HOME}/bin/hbase mapredcp | tr ':' ',') ...</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="paragraph">
<p>The example may not work if you are running HBase from its build directory rather than an installed location.
You may see an error like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>java.lang.RuntimeException: java.lang.ClassNotFoundException: org.apache.hadoop.hbase.mapreduce.RowCounter$RowCounterMapper</pre>
</div>
</div>
<div class="paragraph">
<p>If this occurs, try modifying the command as follows, so that it uses the HBase JARs from the <em>target/</em> directory within the build environment.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=${HBASE_BUILD_HOME}/hbase-mapreduce/target/hbase-mapreduce-VERSION-SNAPSHOT.jar:`${HBASE_BUILD_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_BUILD_HOME}/hbase-mapreduce/target/hbase-mapreduce-VERSION-SNAPSHOT.jar rowcounter usertable</code></pre>
</div>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">Notice to MapReduce users of HBase between 0.96.1 and 0.98.4</div>
<div class="paragraph">
<p>Some MapReduce jobs that use HBase fail to launch.
The symptom is an exception similar to the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Exception in thread "main" java.lang.IllegalAccessError: class
com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass
com.google.protobuf.LiteralByteString
at java.lang.ClassLoader.defineClass1(Native Method)
at java.lang.ClassLoader.defineClass(ClassLoader.java:792)
at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
at java.net.URLClassLoader.defineClass(URLClassLoader.java:449)
at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
at java.security.AccessController.doPrivileged(Native Method)
at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
at
org.apache.hadoop.hbase.protobuf.ProtobufUtil.toScan(ProtobufUtil.java:818)
at
org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.convertScanToString(TableMapReduceUtil.java:433)
at
org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableMapperJob(TableMapReduceUtil.java:186)
at
org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableMapperJob(TableMapReduceUtil.java:147)
at
org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableMapperJob(TableMapReduceUtil.java:270)
at
org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.initTableMapperJob(TableMapReduceUtil.java:100)
...</pre>
</div>
</div>
<div class="paragraph">
<p>This is caused by an optimization introduced in <a href="https://issues.apache.org/jira/browse/HBASE-9867">HBASE-9867</a> that inadvertently introduced a classloader dependency.</p>
</div>
<div class="paragraph">
<p>This affects both jobs using the <code>-libjars</code> option and "fat jar," those which package their runtime dependencies in a nested <code>lib</code> folder.</p>
</div>
<div class="paragraph">
<p>In order to satisfy the new classloader requirements, <code>hbase-protocol.jar</code> must be included in Hadoop&#8217;s classpath.
See <a href="#hbase.mapreduce.classpath">HBase, MapReduce, and the CLASSPATH</a> for current recommendations for resolving classpath errors.
The following is included for historical purposes.</p>
</div>
<div class="paragraph">
<p>This can be resolved system-wide by including a reference to the <code>hbase-protocol.jar</code> in Hadoop&#8217;s lib directory, via a symlink or by copying the jar into the new location.</p>
</div>
<div class="paragraph">
<p>This can also be achieved on a per-job launch basis by including it in the <code>HADOOP_CLASSPATH</code> environment variable at job submission time.
When launching jobs that package their dependencies, all three of the following job launching commands satisfy this requirement:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=/path/to/hbase-protocol.jar:/path/to/hbase/conf hadoop jar MyJob.jar MyJobMainClass
$ HADOOP_CLASSPATH=$(hbase mapredcp):/path/to/hbase/conf hadoop jar MyJob.jar MyJobMainClass
$ HADOOP_CLASSPATH=$(hbase classpath) hadoop jar MyJob.jar MyJobMainClass</code></pre>
</div>
</div>
<div class="paragraph">
<p>For jars that do not package their dependencies, the following command structure is necessary:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=$(hbase mapredcp):/etc/hbase/conf hadoop jar MyApp.jar MyJobMainClass -libjars $(hbase mapredcp | tr ':' ',') ...</code></pre>
</div>
</div>
<div class="paragraph">
<p>See also <a href="https://issues.apache.org/jira/browse/HBASE-10304">HBASE-10304</a> for further discussion of this issue.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_mapreduce_scan_caching"><a class="anchor" href="#_mapreduce_scan_caching"></a>49. MapReduce Scan Caching</h2>
<div class="sectionbody">
<div class="paragraph">
<p>TableMapReduceUtil now restores the option to set scanner caching (the number of rows which are cached before returning the result to the client) on the Scan object that is passed in.
This functionality was lost due to a bug in HBase 0.95 (<a href="https://issues.apache.org/jira/browse/HBASE-11558">HBASE-11558</a>), which is fixed for HBase 0.98.5 and 0.96.3.
The priority order for choosing the scanner caching is as follows:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Caching settings which are set on the scan object.</p>
</li>
<li>
<p>Caching settings which are specified via the configuration option <code>hbase.client.scanner.caching</code>, which can either be set manually in <em>hbase-site.xml</em> or via the helper method <code>TableMapReduceUtil.setScannerCaching()</code>.</p>
</li>
<li>
<p>The default value <code>HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING</code>, which is set to <code>100</code>.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Optimizing the caching settings is a balance between the time the client waits for a result and the number of sets of results the client needs to receive.
If the caching setting is too large, the client could end up waiting for a long time or the request could even time out.
If the setting is too small, the scan needs to return results in several pieces.
If you think of the scan as a shovel, a bigger cache setting is analogous to a bigger shovel, and a smaller cache setting is equivalent to more shoveling in order to fill the bucket.</p>
</div>
<div class="paragraph">
<p>The list of priorities mentioned above allows you to set a reasonable default, and override it for specific operations.</p>
</div>
<div class="paragraph">
<p>See the API documentation for <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> for more details.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_bundled_hbase_mapreduce_jobs"><a class="anchor" href="#_bundled_hbase_mapreduce_jobs"></a>50. Bundled HBase MapReduce Jobs</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The HBase JAR also serves as a Driver for some bundled MapReduce jobs.
To learn about the bundled MapReduce jobs, run the following command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-mapreduce-VERSION.jar
An example program must be given as the first argument.
Valid program names are:
copytable: Export a table from local cluster to peer cluster
completebulkload: Complete a bulk data load.
export: Write table data to HDFS.
import: Import data written by Export.
importtsv: Import data in TSV format.
rowcounter: Count rows in HBase table</code></pre>
</div>
</div>
<div class="paragraph">
<p>Each of the valid program names are bundled MapReduce jobs.
To run one of the jobs, model your command after the following example.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-mapreduce-VERSION.jar rowcounter myTable</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_hbase_as_a_mapreduce_job_data_source_and_data_sink"><a class="anchor" href="#_hbase_as_a_mapreduce_job_data_source_and_data_sink"></a>51. HBase as a MapReduce Job Data Source and Data Sink</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase can be used as a data source, <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html">TableInputFormat</a>, and data sink, <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html">TableOutputFormat</a> or <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.html">MultiTableOutputFormat</a>, for MapReduce jobs.
Writing MapReduce jobs that read or write HBase, it is advisable to subclass <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapper.html">TableMapper</a> and/or <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableReducer.html">TableReducer</a>.
See the do-nothing pass-through classes <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableMapper.html">IdentityTableMapper</a> and <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/IdentityTableReducer.html">IdentityTableReducer</a> for basic usage.
For a more involved example, see <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html">RowCounter</a> or review the <code>org.apache.hadoop.hbase.mapreduce.TestTableMapReduce</code> unit test.</p>
</div>
<div class="paragraph">
<p>If you run MapReduce jobs that use HBase as source or sink, need to specify source and sink table and column names in your configuration.</p>
</div>
<div class="paragraph">
<p>When you read from HBase, the <code>TableInputFormat</code> requests the list of regions from HBase and makes a map, which is either a <code>map-per-region</code> or <code>mapreduce.job.maps</code> map, whichever is smaller.
If your job only has two maps, raise <code>mapreduce.job.maps</code> to a number greater than the number of regions.
Maps will run on the adjacent TaskTracker/NodeManager if you are running a TaskTracer/NodeManager and RegionServer per node.
When writing to HBase, it may make sense to avoid the Reduce step and write back into HBase from within your map.
This approach works when your job does not need the sort and collation that MapReduce does on the map-emitted data.
On insert, HBase 'sorts' so there is no point double-sorting (and shuffling data around your MapReduce cluster) unless you need to.
If you do not need the Reduce, your map might emit counts of records processed for reporting at the end of the job, or set the number of Reduces to zero and use TableOutputFormat.
If running the Reduce step makes sense in your case, you should typically use multiple reducers so that load is spread across the HBase cluster.</p>
</div>
<div class="paragraph">
<p>A new HBase partitioner, the <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.html">HRegionPartitioner</a>, can run as many reducers the number of existing regions.
The HRegionPartitioner is suitable when your table is large and your upload will not greatly alter the number of existing regions upon completion.
Otherwise use the default partitioner.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_writing_hfiles_directly_during_bulk_import"><a class="anchor" href="#_writing_hfiles_directly_during_bulk_import"></a>52. Writing HFiles Directly During Bulk Import</h2>
<div class="sectionbody">
<div class="paragraph">
<p>If you are importing into a new table, you can bypass the HBase API and write your content directly to the filesystem, formatted into HBase data files (HFiles). Your import will run faster, perhaps an order of magnitude faster.
For more on how this mechanism works, see <a href="#arch.bulk.load">Bulk Loading</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_rowcounter_example"><a class="anchor" href="#_rowcounter_example"></a>53. RowCounter Example</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The included <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html">RowCounter</a> MapReduce job uses <code>TableInputFormat</code> and does a count of all rows in the specified table.
To run it, use the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ./bin/hadoop jar hbase-X.X.X.jar</code></pre>
</div>
</div>
<div class="paragraph">
<p>This will invoke the HBase MapReduce Driver class.
Select <code>rowcounter</code> from the choice of jobs offered.
This will print rowcounter usage advice to standard output.
Specify the tablename, column to count, and output directory.
If you have classpath errors, see <a href="#hbase.mapreduce.classpath">HBase, MapReduce, and the CLASSPATH</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="splitter"><a class="anchor" href="#splitter"></a>54. Map-Task Splitting</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="splitter.default"><a class="anchor" href="#splitter.default"></a>54.1. The Default HBase MapReduce Splitter</h3>
<div class="paragraph">
<p>When <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html">TableInputFormat</a> is used to source an HBase table in a MapReduce job, its splitter will make a map task for each region of the table.
Thus, if there are 100 regions in the table, there will be 100 map-tasks for the job - regardless of how many column families are selected in the Scan.</p>
</div>
</div>
<div class="sect2">
<h3 id="splitter.custom"><a class="anchor" href="#splitter.custom"></a>54.2. Custom Splitters</h3>
<div class="paragraph">
<p>For those interested in implementing custom splitters, see the method <code>getSplits</code> in <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.html">TableInputFormatBase</a>.
That is where the logic for map-task assignment resides.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="mapreduce.example"><a class="anchor" href="#mapreduce.example"></a>55. HBase MapReduce Examples</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="mapreduce.example.read"><a class="anchor" href="#mapreduce.example.read"></a>55.1. HBase MapReduce Read Example</h3>
<div class="paragraph">
<p>The following is an example of using HBase as a MapReduce source in read-only manner.
Specifically, there is a Mapper instance but no Reducer, and nothing is being emitted from the Mapper.
The job would be defined as follows&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
Job job = <span class="keyword">new</span> Job(config, <span class="string"><span class="delimiter">&quot;</span><span class="content">ExampleRead</span><span class="delimiter">&quot;</span></span>);
job.setJarByClass(MyReadJob.class); <span class="comment">// class that contains mapper</span>
Scan scan = <span class="keyword">new</span> Scan();
scan.setCaching(<span class="integer">500</span>); <span class="comment">// 1 is the default in Scan, which will be bad for MapReduce jobs</span>
scan.setCacheBlocks(<span class="predefined-constant">false</span>); <span class="comment">// don't set to true for MR jobs</span>
<span class="comment">// set other scan attrs</span>
...
TableMapReduceUtil.initTableMapperJob(
tableName, <span class="comment">// input HBase table name</span>
scan, <span class="comment">// Scan instance to control CF and attribute selection</span>
MyMapper.class, <span class="comment">// mapper</span>
<span class="predefined-constant">null</span>, <span class="comment">// mapper output key</span>
<span class="predefined-constant">null</span>, <span class="comment">// mapper output value</span>
job);
job.setOutputFormatClass(NullOutputFormat.class); <span class="comment">// because we aren't emitting anything from mapper</span>
<span class="type">boolean</span> b = job.waitForCompletion(<span class="predefined-constant">true</span>);
<span class="keyword">if</span> (!b) {
<span class="keyword">throw</span> <span class="keyword">new</span> <span class="exception">IOException</span>(<span class="string"><span class="delimiter">&quot;</span><span class="content">error with job!</span><span class="delimiter">&quot;</span></span>);
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;and the mapper instance would extend <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapper.html">TableMapper</a>&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyMapper</span> <span class="directive">extends</span> TableMapper&lt;Text, Text&gt; {
<span class="directive">public</span> <span class="type">void</span> map(ImmutableBytesWritable row, <span class="predefined-type">Result</span> value, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">InterruptedException</span>, <span class="exception">IOException</span> {
<span class="comment">// process data for the row from the Result instance.</span>
}
}</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.readwrite"><a class="anchor" href="#mapreduce.example.readwrite"></a>55.2. HBase MapReduce Read/Write Example</h3>
<div class="paragraph">
<p>The following is an example of using HBase both as a source and as a sink with MapReduce.
This example will simply copy data from one table to another.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
Job job = <span class="keyword">new</span> Job(config,<span class="string"><span class="delimiter">&quot;</span><span class="content">ExampleReadWrite</span><span class="delimiter">&quot;</span></span>);
job.setJarByClass(MyReadWriteJob.class); <span class="comment">// class that contains mapper</span>
Scan scan = <span class="keyword">new</span> Scan();
scan.setCaching(<span class="integer">500</span>); <span class="comment">// 1 is the default in Scan, which will be bad for MapReduce jobs</span>
scan.setCacheBlocks(<span class="predefined-constant">false</span>); <span class="comment">// don't set to true for MR jobs</span>
<span class="comment">// set other scan attrs</span>
TableMapReduceUtil.initTableMapperJob(
sourceTable, <span class="comment">// input table</span>
scan, <span class="comment">// Scan instance to control CF and attribute selection</span>
MyMapper.class, <span class="comment">// mapper class</span>
<span class="predefined-constant">null</span>, <span class="comment">// mapper output key</span>
<span class="predefined-constant">null</span>, <span class="comment">// mapper output value</span>
job);
TableMapReduceUtil.initTableReducerJob(
targetTable, <span class="comment">// output table</span>
<span class="predefined-constant">null</span>, <span class="comment">// reducer class</span>
job);
job.setNumReduceTasks(<span class="integer">0</span>);
<span class="type">boolean</span> b = job.waitForCompletion(<span class="predefined-constant">true</span>);
<span class="keyword">if</span> (!b) {
<span class="keyword">throw</span> <span class="keyword">new</span> <span class="exception">IOException</span>(<span class="string"><span class="delimiter">&quot;</span><span class="content">error with job!</span><span class="delimiter">&quot;</span></span>);
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>An explanation is required of what <code>TableMapReduceUtil</code> is doing, especially with the reducer. <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html">TableOutputFormat</a> is being used as the outputFormat class, and several parameters are being set on the config (e.g., <code>TableOutputFormat.OUTPUT_TABLE</code>), as well as setting the reducer output key to <code>ImmutableBytesWritable</code> and reducer value to <code>Writable</code>.
These could be set by the programmer on the job and conf, but <code>TableMapReduceUtil</code> tries to make things easier.</p>
</div>
<div class="paragraph">
<p>The following is the example mapper, which will create a <code>Put</code> and matching the input <code>Result</code> and emit it.
Note: this is what the CopyTable utility does.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyMapper</span> <span class="directive">extends</span> TableMapper&lt;ImmutableBytesWritable, Put&gt; {
<span class="directive">public</span> <span class="type">void</span> map(ImmutableBytesWritable row, <span class="predefined-type">Result</span> value, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="comment">// this example is just copying the data from the source table...</span>
context.write(row, resultToPut(row,value));
}
<span class="directive">private</span> <span class="directive">static</span> Put resultToPut(ImmutableBytesWritable key, <span class="predefined-type">Result</span> result) <span class="directive">throws</span> <span class="exception">IOException</span> {
Put put = <span class="keyword">new</span> Put(key.get());
<span class="keyword">for</span> (Cell cell : result.listCells()) {
put.add(cell);
}
<span class="keyword">return</span> put;
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>There isn&#8217;t actually a reducer step, so <code>TableOutputFormat</code> takes care of sending the <code>Put</code> to the target table.</p>
</div>
<div class="paragraph">
<p>This is just an example, developers could choose not to use <code>TableOutputFormat</code> and connect to the target table themselves.</p>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.readwrite.multi"><a class="anchor" href="#mapreduce.example.readwrite.multi"></a>55.3. HBase MapReduce Read/Write Example With Multi-Table Output</h3>
<div class="paragraph">
<p>TODO: example for <code>MultiTableOutputFormat</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.summary"><a class="anchor" href="#mapreduce.example.summary"></a>55.4. HBase MapReduce Summary to HBase Example</h3>
<div class="paragraph">
<p>The following example uses HBase as a MapReduce source and sink with a summarization step.
This example will count the number of distinct instances of a value in a table and write those summarized counts in another table.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
Job job = <span class="keyword">new</span> Job(config,<span class="string"><span class="delimiter">&quot;</span><span class="content">ExampleSummary</span><span class="delimiter">&quot;</span></span>);
job.setJarByClass(MySummaryJob.class); <span class="comment">// class that contains mapper and reducer</span>
Scan scan = <span class="keyword">new</span> Scan();
scan.setCaching(<span class="integer">500</span>); <span class="comment">// 1 is the default in Scan, which will be bad for MapReduce jobs</span>
scan.setCacheBlocks(<span class="predefined-constant">false</span>); <span class="comment">// don't set to true for MR jobs</span>
<span class="comment">// set other scan attrs</span>
TableMapReduceUtil.initTableMapperJob(
sourceTable, <span class="comment">// input table</span>
scan, <span class="comment">// Scan instance to control CF and attribute selection</span>
MyMapper.class, <span class="comment">// mapper class</span>
Text.class, <span class="comment">// mapper output key</span>
IntWritable.class, <span class="comment">// mapper output value</span>
job);
TableMapReduceUtil.initTableReducerJob(
targetTable, <span class="comment">// output table</span>
MyTableReducer.class, <span class="comment">// reducer class</span>
job);
job.setNumReduceTasks(<span class="integer">1</span>); <span class="comment">// at least one, adjust as required</span>
<span class="type">boolean</span> b = job.waitForCompletion(<span class="predefined-constant">true</span>);
<span class="keyword">if</span> (!b) {
<span class="keyword">throw</span> <span class="keyword">new</span> <span class="exception">IOException</span>(<span class="string"><span class="delimiter">&quot;</span><span class="content">error with job!</span><span class="delimiter">&quot;</span></span>);
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>In this example mapper a column with a String-value is chosen as the value to summarize upon.
This value is used as the key to emit from the mapper, and an <code>IntWritable</code> represents an instance counter.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyMapper</span> <span class="directive">extends</span> TableMapper&lt;Text, IntWritable&gt; {
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR1 = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr1</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">private</span> <span class="directive">final</span> IntWritable ONE = <span class="keyword">new</span> IntWritable(<span class="integer">1</span>);
<span class="directive">private</span> Text text = <span class="keyword">new</span> Text();
<span class="directive">public</span> <span class="type">void</span> map(ImmutableBytesWritable row, <span class="predefined-type">Result</span> value, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="predefined-type">String</span> val = <span class="keyword">new</span> <span class="predefined-type">String</span>(value.getValue(CF, ATTR1));
text.set(val); <span class="comment">// we can only emit Writables...</span>
context.write(text, ONE);
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>In the reducer, the "ones" are counted (just like any other MR example that does this), and then emits a <code>Put</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyTableReducer</span> <span class="directive">extends</span> TableReducer&lt;Text, IntWritable, ImmutableBytesWritable&gt; {
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> COUNT = <span class="string"><span class="delimiter">&quot;</span><span class="content">count</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="type">void</span> reduce(Text key, <span class="predefined-type">Iterable</span>&lt;IntWritable&gt; values, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="type">int</span> i = <span class="integer">0</span>;
<span class="keyword">for</span> (IntWritable val : values) {
i += val.get();
}
Put put = <span class="keyword">new</span> Put(Bytes.toBytes(key.toString()));
put.add(CF, COUNT, Bytes.toBytes(i));
context.write(<span class="predefined-constant">null</span>, put);
}
}</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.summary.file"><a class="anchor" href="#mapreduce.example.summary.file"></a>55.5. HBase MapReduce Summary to File Example</h3>
<div class="paragraph">
<p>This very similar to the summary example above, with exception that this is using HBase as a MapReduce source but HDFS as the sink.
The differences are in the job setup and in the reducer.
The mapper remains the same.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
Job job = <span class="keyword">new</span> Job(config,<span class="string"><span class="delimiter">&quot;</span><span class="content">ExampleSummaryToFile</span><span class="delimiter">&quot;</span></span>);
job.setJarByClass(MySummaryFileJob.class); <span class="comment">// class that contains mapper and reducer</span>
Scan scan = <span class="keyword">new</span> Scan();
scan.setCaching(<span class="integer">500</span>); <span class="comment">// 1 is the default in Scan, which will be bad for MapReduce jobs</span>
scan.setCacheBlocks(<span class="predefined-constant">false</span>); <span class="comment">// don't set to true for MR jobs</span>
<span class="comment">// set other scan attrs</span>
TableMapReduceUtil.initTableMapperJob(
sourceTable, <span class="comment">// input table</span>
scan, <span class="comment">// Scan instance to control CF and attribute selection</span>
MyMapper.class, <span class="comment">// mapper class</span>
Text.class, <span class="comment">// mapper output key</span>
IntWritable.class, <span class="comment">// mapper output value</span>
job);
job.setReducerClass(MyReducer.class); <span class="comment">// reducer class</span>
job.setNumReduceTasks(<span class="integer">1</span>); <span class="comment">// at least one, adjust as required</span>
FileOutputFormat.setOutputPath(job, <span class="keyword">new</span> Path(<span class="string"><span class="delimiter">&quot;</span><span class="content">/tmp/mr/mySummaryFile</span><span class="delimiter">&quot;</span></span>)); <span class="comment">// adjust directories as required</span>
<span class="type">boolean</span> b = job.waitForCompletion(<span class="predefined-constant">true</span>);
<span class="keyword">if</span> (!b) {
<span class="keyword">throw</span> <span class="keyword">new</span> <span class="exception">IOException</span>(<span class="string"><span class="delimiter">&quot;</span><span class="content">error with job!</span><span class="delimiter">&quot;</span></span>);
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>As stated above, the previous Mapper can run unchanged with this example.
As for the Reducer, it is a "generic" Reducer instead of extending TableMapper and emitting Puts.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyReducer</span> <span class="directive">extends</span> Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
<span class="directive">public</span> <span class="type">void</span> reduce(Text key, <span class="predefined-type">Iterable</span>&lt;IntWritable&gt; values, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="type">int</span> i = <span class="integer">0</span>;
<span class="keyword">for</span> (IntWritable val : values) {
i += val.get();
}
context.write(key, <span class="keyword">new</span> IntWritable(i));
}
}</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.summary.noreducer"><a class="anchor" href="#mapreduce.example.summary.noreducer"></a>55.6. HBase MapReduce Summary to HBase Without Reducer</h3>
<div class="paragraph">
<p>It is also possible to perform summaries without a reducer - if you use HBase as the reducer.</p>
</div>
<div class="paragraph">
<p>An HBase target table would need to exist for the job summary.
The Table method <code>incrementColumnValue</code> would be used to atomically increment values.
From a performance perspective, it might make sense to keep a Map of values with their values to be incremented for each map-task, and make one update per key at during the <code>cleanup</code> method of the mapper.
However, your mileage may vary depending on the number of rows to be processed and unique keys.</p>
</div>
<div class="paragraph">
<p>In the end, the summary results are in HBase.</p>
</div>
</div>
<div class="sect2">
<h3 id="mapreduce.example.summary.rdbms"><a class="anchor" href="#mapreduce.example.summary.rdbms"></a>55.7. HBase MapReduce Summary to RDBMS</h3>
<div class="paragraph">
<p>Sometimes it is more appropriate to generate summaries to an RDBMS.
For these cases, it is possible to generate summaries directly to an RDBMS via a custom reducer.
The <code>setup</code> method can connect to an RDBMS (the connection information can be passed via custom parameters in the context) and the cleanup method can close the connection.</p>
</div>
<div class="paragraph">
<p>It is critical to understand that number of reducers for the job affects the summarization implementation, and you&#8217;ll have to design this into your reducer.
Specifically, whether it is designed to run as a singleton (one reducer) or multiple reducers.
Neither is right or wrong, it depends on your use-case.
Recognize that the more reducers that are assigned to the job, the more simultaneous connections to the RDBMS will be created - this will scale, but only to a point.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">class</span> <span class="class">MyRdbmsReducer</span> <span class="directive">extends</span> Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
<span class="directive">private</span> <span class="predefined-type">Connection</span> c = <span class="predefined-constant">null</span>;
<span class="directive">public</span> <span class="type">void</span> setup(<span class="predefined-type">Context</span> context) {
<span class="comment">// create DB connection...</span>
}
<span class="directive">public</span> <span class="type">void</span> reduce(Text key, <span class="predefined-type">Iterable</span>&lt;IntWritable&gt; values, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="comment">// do summarization</span>
<span class="comment">// in this example the keys are Text, but this is just an example</span>
}
<span class="directive">public</span> <span class="type">void</span> cleanup(<span class="predefined-type">Context</span> context) {
<span class="comment">// close db connection</span>
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>In the end, the summary results are written to your RDBMS table/s.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="mapreduce.htable.access"><a class="anchor" href="#mapreduce.htable.access"></a>56. Accessing Other HBase Tables in a MapReduce Job</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Although the framework currently allows one HBase table as input to a MapReduce job, other HBase tables can be accessed as lookup tables, etc., in a MapReduce job via creating an Table instance in the setup method of the Mapper.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">MyMapper</span> <span class="directive">extends</span> TableMapper&lt;Text, LongWritable&gt; {
<span class="directive">private</span> Table myOtherTable;
<span class="directive">public</span> <span class="type">void</span> setup(<span class="predefined-type">Context</span> context) {
<span class="comment">// In here create a Connection to the cluster and save it or use the Connection</span>
<span class="comment">// from the existing table</span>
myOtherTable = connection.getTable(<span class="string"><span class="delimiter">&quot;</span><span class="content">myOtherTable</span><span class="delimiter">&quot;</span></span>);
}
<span class="directive">public</span> <span class="type">void</span> map(ImmutableBytesWritable row, <span class="predefined-type">Result</span> value, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="comment">// process Result...</span>
<span class="comment">// use 'myOtherTable' for lookups</span>
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="mapreduce.specex"><a class="anchor" href="#mapreduce.specex"></a>57. Speculative Execution</h2>
<div class="sectionbody">
<div class="paragraph">
<p>It is generally advisable to turn off speculative execution for MapReduce jobs that use HBase as a source.
This can either be done on a per-Job basis through properties, or on the entire cluster.
Especially for longer running jobs, speculative execution will create duplicate map-tasks which will double-write your data to HBase; this is probably not what you want.</p>
</div>
<div class="paragraph">
<p>See <a href="#spec.ex">spec.ex</a> for more information.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="cascading"><a class="anchor" href="#cascading"></a>58. Cascading</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="http://www.cascading.org/">Cascading</a> is an alternative API for MapReduce, which
actually uses MapReduce, but allows you to write your MapReduce code in a simplified
way.</p>
</div>
<div class="paragraph">
<p>The following example shows a Cascading <code>Flow</code> which "sinks" data into an HBase cluster. The same
<code>hBaseTap</code> API could be used to "source" data as well.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">// read data from the default filesystem</span>
<span class="comment">// emits two fields: &quot;offset&quot; and &quot;line&quot;</span>
Tap source = <span class="keyword">new</span> Hfs( <span class="keyword">new</span> TextLine(), inputFileLhs );
<span class="comment">// store data in an HBase cluster</span>
<span class="comment">// accepts fields &quot;num&quot;, &quot;lower&quot;, and &quot;upper&quot;</span>
<span class="comment">// will automatically scope incoming fields to their proper familyname, &quot;left&quot; or &quot;right&quot;</span>
Fields keyFields = <span class="keyword">new</span> Fields( <span class="string"><span class="delimiter">&quot;</span><span class="content">num</span><span class="delimiter">&quot;</span></span> );
<span class="predefined-type">String</span><span class="type">[]</span> familyNames = {<span class="string"><span class="delimiter">&quot;</span><span class="content">left</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">right</span><span class="delimiter">&quot;</span></span>};
Fields<span class="type">[]</span> valueFields = <span class="keyword">new</span> Fields<span class="type">[]</span> {<span class="keyword">new</span> Fields( <span class="string"><span class="delimiter">&quot;</span><span class="content">lower</span><span class="delimiter">&quot;</span></span> ), <span class="keyword">new</span> Fields( <span class="string"><span class="delimiter">&quot;</span><span class="content">upper</span><span class="delimiter">&quot;</span></span> ) };
Tap hBaseTap = <span class="keyword">new</span> HBaseTap( <span class="string"><span class="delimiter">&quot;</span><span class="content">multitable</span><span class="delimiter">&quot;</span></span>, <span class="keyword">new</span> HBaseScheme( keyFields, familyNames, valueFields ), SinkMode.REPLACE );
<span class="comment">// a simple pipe assembly to parse the input into fields</span>
<span class="comment">// a real app would likely chain multiple Pipes together for more complex processing</span>
<span class="predefined-type">Pipe</span> parsePipe = <span class="keyword">new</span> Each( <span class="string"><span class="delimiter">&quot;</span><span class="content">insert</span><span class="delimiter">&quot;</span></span>, <span class="keyword">new</span> Fields( <span class="string"><span class="delimiter">&quot;</span><span class="content">line</span><span class="delimiter">&quot;</span></span> ), <span class="keyword">new</span> RegexSplitter( <span class="keyword">new</span> Fields( <span class="string"><span class="delimiter">&quot;</span><span class="content">num</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">lower</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">upper</span><span class="delimiter">&quot;</span></span> ), <span class="string"><span class="delimiter">&quot;</span><span class="content"> </span><span class="delimiter">&quot;</span></span> ) );
<span class="comment">// &quot;plan&quot; a cluster executable Flow</span>
<span class="comment">// this connects the source Tap and hBaseTap (the sink Tap) to the parsePipe</span>
Flow parseFlow = <span class="keyword">new</span> FlowConnector( properties ).connect( source, hBaseTap, parsePipe );
<span class="comment">// start the flow, and block until complete</span>
parseFlow.complete();
<span class="comment">// open an iterator on the HBase table we stuffed data into</span>
TupleEntryIterator iterator = parseFlow.openSink();
<span class="keyword">while</span>(iterator.hasNext())
{
<span class="comment">// print out each tuple from HBase</span>
<span class="predefined-type">System</span>.out.println( <span class="string"><span class="delimiter">&quot;</span><span class="content">iterator.next() = </span><span class="delimiter">&quot;</span></span> + iterator.next() );
}
iterator.close();</code></pre>
</div>
</div>
</div>
</div>
<h1 id="security" class="sect0"><a class="anchor" href="#security"></a>Securing Apache HBase</h1>
<div class="openblock partintro">
<div class="content">
<div class="admonitionblock important">
<table>
<tr>
<td class="icon">
<i class="fa icon-important" title="Important"></i>
</td>
<td class="content">
<div class="title">Reporting Security Bugs</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
To protect existing HBase installations from exploitation, please <strong>do not</strong> use JIRA to report security-related bugs. Instead, send your report to the mailing list <a href="mailto:private@hbase.apache.org">private@hbase.apache.org</a>, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>HBase adheres to the Apache Software Foundation&#8217;s policy on reported vulnerabilities, available at <a href="http://apache.org/security/" class="bare">http://apache.org/security/</a>.</p>
</div>
<div class="paragraph">
<p>If you wish to send an encrypted report, you can use the GPG details provided for the general ASF security list. This will likely increase the response time to your report.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>HBase provides mechanisms to secure various components and aspects of HBase and how it relates to the rest of the Hadoop infrastructure, as well as clients and resources outside Hadoop.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_using_secure_http_https_for_the_web_ui"><a class="anchor" href="#_using_secure_http_https_for_the_web_ui"></a>59. Using Secure HTTP (HTTPS) for the Web UI</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A default HBase install uses insecure HTTP connections for Web UIs for the master and region servers.
To enable secure HTTP (HTTPS) connections instead, set <code>hbase.ssl.enabled</code> to <code>true</code> in <em>hbase-site.xml</em>.
This does not change the port used by the Web UI.
To change the port for the web UI for a given HBase component, configure that port&#8217;s setting in hbase-site.xml.
These settings are:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.master.info.port</code></p>
</li>
<li>
<p><code>hbase.regionserver.info.port</code></p>
</li>
</ul>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">If you enable HTTPS, clients should avoid using the non-secure HTTP connection.</div>
<div class="paragraph">
<p>If you enable secure HTTP, clients should connect to HBase using the <code>https://</code> URL.
Clients using the <code>http://</code> URL will receive an HTTP response of <code>200</code>, but will not receive any data.
The following exception is logged:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>javax.net.ssl.SSLException: Unrecognized SSL message, plaintext connection?</pre>
</div>
</div>
<div class="paragraph">
<p>This is because the same port is used for HTTP and HTTPS.</p>
</div>
<div class="paragraph">
<p>HBase uses Jetty for the Web UI.
Without modifying Jetty itself, it does not seem possible to configure Jetty to redirect one port to another on the same host.
See Nick Dimiduk&#8217;s contribution on this <a href="http://stackoverflow.com/questions/20611815/redirect-from-http-to-https-in-jetty">Stack Overflow</a> thread for more information.
If you know how to fix this without opening a second port for HTTPS, patches are appreciated.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.secure.spnego.ui"><a class="anchor" href="#hbase.secure.spnego.ui"></a>60. Using SPNEGO for Kerberos authentication with Web UIs</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Kerberos-authentication to HBase Web UIs can be enabled via configuring SPNEGO with the <code>hbase.security.authentication.ui</code>
property in <em>hbase-site.xml</em>. Enabling this authentication requires that HBase is also configured to use Kerberos authentication
for RPCs (e.g <code>hbase.security.authentication</code> = <code>kerberos</code>).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication.ui<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Controls what kind of authentication should be used for the HBase web UIs.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>The Kerberos keytab file to use for SPNEGO authentication by the web server.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>A number of properties exist to configure SPNEGO authentication for the web server:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication.spnego.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>HTTP/_HOST@EXAMPLE.COM<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Required for SPNEGO, the Kerberos principal to use for SPNEGO authentication by the
web server. The _HOST keyword will be automatically substituted with the node's
hostname.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication.spnego.kerberos.keytab<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/etc/security/keytabs/spnego.service.keytab<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Required for SPNEGO, the Kerberos keytab file to use for SPNEGO authentication by the
web server.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication.spnego.kerberos.name.rules<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span><span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Optional, Hadoop-style `auth_to_local` rules which will be parsed and used in the
handling of Kerberos principals<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication.signature.secret.file<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span><span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>Optional, a file whose contents will be used as a secret to sign the HTTP cookies
as a part of the SPNEGO authentication handshake. If this is not provided, Java's `Random` library
will be used for the secret.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.secure.configuration"><a class="anchor" href="#hbase.secure.configuration"></a>61. Secure Client Access to Apache HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Newer releases of Apache HBase (&gt;= 0.92) support optional SASL authentication of clients.
See also Matteo Bertozzi&#8217;s article on <a href="https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/">Understanding User Authentication and Authorization in Apache HBase</a>.</p>
</div>
<div class="paragraph">
<p>This describes how to set up Apache HBase and clients for connection to secure HBase resources.</p>
</div>
<div class="sect2">
<h3 id="security.prerequisites"><a class="anchor" href="#security.prerequisites"></a>61.1. Prerequisites</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">Hadoop Authentication Configuration</dt>
<dd>
<p>To run HBase RPC with strong authentication, you must set <code>hbase.security.authentication</code> to <code>kerberos</code>.
In this case, you must also set <code>hadoop.security.authentication</code> to <code>kerberos</code> in core-site.xml.
Otherwise, you would be using strong authentication for HBase but not for the underlying HDFS, which would cancel out any benefit.</p>
</dd>
<dt class="hdlist1">Kerberos KDC</dt>
<dd>
<p>You need to have a working Kerberos KDC.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_server_side_configuration_for_secure_operation"><a class="anchor" href="#_server_side_configuration_for_secure_operation"></a>61.2. Server-side Configuration for Secure Operation</h3>
<div class="paragraph">
<p>First, refer to <a href="#security.prerequisites">security.prerequisites</a> and ensure that your underlying HDFS configuration is secure.</p>
</div>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file on every server machine in the cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.token.TokenProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>A full shutdown and restart of HBase service is required when deploying these configuration changes.</p>
</div>
</div>
<div class="sect2">
<h3 id="_client_side_configuration_for_secure_operation"><a class="anchor" href="#_client_side_configuration_for_secure_operation"></a>61.3. Client-side Configuration for Secure Operation</h3>
<div class="paragraph">
<p>First, refer to <a href="#security.prerequisites">Prerequisites</a> and ensure that your underlying HDFS configuration is secure.</p>
</div>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file on every client:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Before 2.2.0 version, the client environment must be logged in to Kerberos from KDC or keytab via the <code>kinit</code> command before communication with the HBase cluster will be possible.</p>
</div>
<div class="paragraph">
<p>Since 2.2.0, client can specify the following configurations in <code>hbase-site.xml</code>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.keytab.file<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/local/path/to/client/keytab<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.keytab.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>foo@EXAMPLE.COM<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Then application can automatically do the login and credential renewal jobs without client interference.</p>
</div>
<div class="paragraph">
<p>It&#8217;s optional feature, client, who upgrades to 2.2.0, can still keep their login and credential renewal logic already did in older version, as long as keeping <code>hbase.client.keytab.file</code>
and <code>hbase.client.keytab.principal</code> are unset.</p>
</div>
<div class="paragraph">
<p>Be advised that if the <code>hbase.security.authentication</code> in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.</p>
</div>
<div class="paragraph">
<p>Once HBase is configured for secure RPC it is possible to optionally configure encrypted communication.
To do so, add the following to the <code>hbase-site.xml</code> file on every client:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rpc.protection<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>privacy<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>This configuration property can also be set on a per-connection basis.
Set it in the <code>Configuration</code> supplied to <code>Table</code>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
conf.set(<span class="string"><span class="delimiter">&quot;</span><span class="content">hbase.rpc.protection</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">privacy</span><span class="delimiter">&quot;</span></span>);
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TableName.valueOf(tablename))) {
.... do your stuff
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Expect a ~10% performance penalty for encrypted communication.</p>
</div>
</div>
<div class="sect2">
<h3 id="security.client.thrift"><a class="anchor" href="#security.client.thrift"></a>61.4. Client-side Configuration for Secure Operation - Thrift Gateway</h3>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file for every Thrift gateway:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.thrift.keytab.file<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/etc/hbase/conf/hbase.keytab<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.thrift.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$USER/_HOST@HADOOP.LOCALDOMAIN<span class="tag">&lt;/value&gt;</span>
<span class="comment">&lt;!-- TODO: This may need to be HTTP/_HOST@&lt;REALM&gt; and _HOST may not work.
You may have to put the concrete full hostname.
--&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Add these if you need to configure a different DNS interface from the default --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.thrift.dns.interface<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.thrift.dns.nameserver<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Substitute the appropriate credential and keytab for <em>$USER</em> and <em>$KEYTAB</em> respectively.</p>
</div>
<div class="paragraph">
<p>In order to use the Thrift API principal to interact with HBase, it is also necessary to add the <code>hbase.thrift.kerberos.principal</code> to the <code><em>acl</em></code> table.
For example, to give the Thrift API principal, <code>thrift_server</code>, administrative access, a command such as this one will suffice:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">grant</span> <span class="string"><span class="delimiter">'</span><span class="content">thrift_server</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">RWCA</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information about ACLs, please see the <a href="#hbase.accesscontrol.configuration">Access Control Labels (ACLs)</a> section</p>
</div>
<div class="paragraph">
<p>The Thrift gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the Thrift gateway itself.
All client access via the Thrift gateway will use the Thrift gateway&#8217;s credential and have its privilege.</p>
</div>
</div>
<div class="sect2">
<h3 id="security.gateway.thrift"><a class="anchor" href="#security.gateway.thrift"></a>61.5. Configure the Thrift Gateway to Authenticate on Behalf of the Client</h3>
<div class="paragraph">
<p><a href="#security.client.thrift">Client-side Configuration for Secure Operation - Thrift Gateway</a> describes how to authenticate a Thrift client to HBase using a fixed user.
As an alternative, you can configure the Thrift gateway to authenticate to HBase on the client&#8217;s behalf, and to access HBase using a proxy user.
This was implemented in <a href="https://issues.apache.org/jira/browse/HBASE-11349">HBASE-11349</a> for Thrift 1, and <a href="https://issues.apache.org/jira/browse/HBASE-11474">HBASE-11474</a> for Thrift 2.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Limitations with Thrift Framed Transport</div>
<div class="paragraph">
<p>If you use framed transport, you cannot yet take advantage of this feature, because SASL does not work with Thrift framed transport at this time.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>To enable it, do the following.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Be sure Thrift is running in secure mode, by following the procedure described in <a href="#security.client.thrift">Client-side Configuration for Secure Operation - Thrift Gateway</a>.</p>
</li>
<li>
<p>Be sure that HBase is configured to allow proxy users, as described in <a href="#security.rest.gateway">REST Gateway Impersonation Configuration</a>.</p>
</li>
<li>
<p>In <em>hbase-site.xml</em> for each cluster node running a Thrift gateway, set the property <code>hbase.thrift.security.qop</code> to one of the following three values:</p>
<div class="ulist">
<ul>
<li>
<p><code>privacy</code> - authentication, integrity, and confidentiality checking.</p>
</li>
<li>
<p><code>integrity</code> - authentication and integrity checking</p>
</li>
<li>
<p><code>authentication</code> - authentication checking only</p>
</li>
</ul>
</div>
</li>
<li>
<p>Restart the Thrift gateway processes for the changes to take effect.
If a node is running Thrift, the output of the <code>jps</code> command will list a <code>ThriftServer</code> process.
To stop Thrift on a node, run the command <code>bin/hbase-daemon.sh stop thrift</code>.
To start Thrift on a node, run the command <code>bin/hbase-daemon.sh start thrift</code>.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="security.gateway.thrift.doas"><a class="anchor" href="#security.gateway.thrift.doas"></a>61.6. Configure the Thrift Gateway to Use the <code>doAs</code> Feature</h3>
<div class="paragraph">
<p><a href="#security.gateway.thrift">Configure the Thrift Gateway to Authenticate on Behalf of the Client</a> describes how to configure the Thrift gateway to authenticate to HBase on the client&#8217;s behalf, and to access HBase using a proxy user. The limitation of this approach is that after the client is initialized with a particular set of credentials, it cannot change these credentials during the session. The <code>doAs</code> feature provides a flexible way to impersonate multiple principals using the same client. This feature was implemented in <a href="https://issues.apache.org/jira/browse/HBASE-12640">HBASE-12640</a> for Thrift 1, but is currently not available for Thrift 2.</p>
</div>
<div class="paragraph">
<p><strong>To enable the <code>doAs</code> feature</strong>, add the following to the <em>hbase-site.xml</em> file for every Thrift gateway:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.thrift.http<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.thrift.support.proxyuser<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true/value<span class="error">&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p><strong>To allow proxy users</strong> when using <code>doAs</code> impersonation, add the following to the <em>hbase-site.xml</em> file for every HBase node:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.proxyuser.$USER.groups<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$GROUPS<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.proxyuser.$USER.hosts<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$GROUPS<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Take a look at the
<a href="https://github.com/apache/hbase/blob/master/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java">demo client</a>
to get an overall idea of how to use this feature in your client.</p>
</div>
</div>
<div class="sect2">
<h3 id="_client_side_configuration_for_secure_operation_rest_gateway"><a class="anchor" href="#_client_side_configuration_for_secure_operation_rest_gateway"></a>61.7. Client-side Configuration for Secure Operation - REST Gateway</h3>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file for every REST gateway:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.keytab.file<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$KEYTAB<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$USER/_HOST@HADOOP.LOCALDOMAIN<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Substitute the appropriate credential and keytab for <em>$USER</em> and <em>$KEYTAB</em> respectively.</p>
</div>
<div class="paragraph">
<p>The REST gateway will authenticate with HBase using the supplied credential.</p>
</div>
<div class="paragraph">
<p>In order to use the REST API principal to interact with HBase, it is also necessary to add the <code>hbase.rest.kerberos.principal</code> to the <code><em>acl</em></code> table.
For example, to give the REST API principal, <code>rest_server</code>, administrative access, a command such as this one will suffice:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">grant</span> <span class="string"><span class="delimiter">'</span><span class="content">rest_server</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">RWCA</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information about ACLs, please see the <a href="#hbase.accesscontrol.configuration">Access Control Labels (ACLs)</a> section</p>
</div>
<div class="paragraph">
<p>HBase REST gateway supports <a href="https://hadoop.apache.org/docs/stable/hadoop-auth/index.html">SPNEGO HTTP authentication</a> for client access to the gateway.
To enable REST gateway Kerberos authentication for client access, add the following to the <code>hbase-site.xml</code> file for every REST gateway.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.support.proxyuser<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.type<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>HTTP/_HOST@HADOOP.LOCALDOMAIN<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.kerberos.keytab<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$KEYTAB<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Add these if you need to configure a different DNS interface from the default --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.dns.interface<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.dns.nameserver<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Substitute the keytab for HTTP for <em>$KEYTAB</em>.</p>
</div>
<div class="paragraph">
<p>HBase REST gateway supports different 'hbase.rest.authentication.type': simple, kerberos.
You can also implement a custom authentication by implementing Hadoop AuthenticationHandler, then specify the full class name as 'hbase.rest.authentication.type' value.
For more information, refer to <a href="https://hadoop.apache.org/docs/stable/hadoop-auth/index.html">SPNEGO HTTP authentication</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="security.rest.gateway"><a class="anchor" href="#security.rest.gateway"></a>61.8. REST Gateway Impersonation Configuration</h3>
<div class="paragraph">
<p>By default, the REST gateway doesn&#8217;t support impersonation.
It accesses the HBase on behalf of clients as the user configured as in the previous section.
To the HBase server, all requests are from the REST gateway user.
The actual users are unknown.
You can turn on the impersonation support.
With impersonation, the REST gateway user is a proxy user.
The HBase server knows the actual/real user of each request.
So it can apply proper authorizations.</p>
</div>
<div class="paragraph">
<p>To turn on REST gateway impersonation, we need to configure HBase servers (masters and region servers) to allow proxy users; configure REST gateway to enable impersonation.</p>
</div>
<div class="paragraph">
<p>To allow proxy users, add the following to the <code>hbase-site.xml</code> file for every HBase server:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.proxyuser.$USER.groups<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$GROUPS<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.proxyuser.$USER.hosts<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$GROUPS<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Substitute the REST gateway proxy user for <em>$USER</em>, and the allowed group list for <em>$GROUPS</em>.</p>
</div>
<div class="paragraph">
<p>To enable REST gateway impersonation, add the following to the <code>hbase-site.xml</code> file for every REST gateway.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.type<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>kerberos<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>HTTP/_HOST@HADOOP.LOCALDOMAIN<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rest.authentication.kerberos.keytab<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$KEYTAB<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Substitute the keytab for HTTP for <em>$KEYTAB</em>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.secure.simpleconfiguration"><a class="anchor" href="#hbase.secure.simpleconfiguration"></a>62. Simple User Access to Apache HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Newer releases of Apache HBase (&gt;= 0.92) support optional SASL authentication of clients.
See also Matteo Bertozzi&#8217;s article on <a href="https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/">Understanding User Authentication and Authorization in Apache HBase</a>.</p>
</div>
<div class="paragraph">
<p>This describes how to set up Apache HBase and clients for simple user access to HBase resources.</p>
</div>
<div class="sect2">
<h3 id="_simple_versus_secure_access"><a class="anchor" href="#_simple_versus_secure_access"></a>62.1. Simple versus Secure Access</h3>
<div class="paragraph">
<p>The following section shows how to set up simple user access.
Simple user access is not a secure method of operating HBase.
This method is used to prevent users from making mistakes.
It can be used to mimic the Access Control using on a development system without having to set up Kerberos.</p>
</div>
<div class="paragraph">
<p>This method is not used to prevent malicious or hacking attempts.
To make HBase secure against these types of attacks, you must configure HBase for secure operation.
Refer to the section <a href="#hbase.secure.configuration">Secure Client Access to Apache HBase</a> and complete all of the steps described there.</p>
</div>
</div>
<div class="sect2">
<h3 id="_prerequisites"><a class="anchor" href="#_prerequisites"></a>62.2. Prerequisites</h3>
<div class="paragraph">
<p>None</p>
</div>
</div>
<div class="sect2">
<h3 id="_server_side_configuration_for_simple_user_access_operation"><a class="anchor" href="#_server_side_configuration_for_simple_user_access_operation"></a>62.3. Server-side Configuration for Simple User Access Operation</h3>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file on every server machine in the cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>simple<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.regionserver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For 0.94, add the following to the <code>hbase-site.xml</code> file on every server machine in the cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rpc.engine<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.ipc.SecureRpcEngine<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>A full shutdown and restart of HBase service is required when deploying these configuration changes.</p>
</div>
</div>
<div class="sect2">
<h3 id="_client_side_configuration_for_simple_user_access_operation"><a class="anchor" href="#_client_side_configuration_for_simple_user_access_operation"></a>62.4. Client-side Configuration for Simple User Access Operation</h3>
<div class="paragraph">
<p>Add the following to the <code>hbase-site.xml</code> file on every client:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authentication<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>simple<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For 0.94, add the following to the <code>hbase-site.xml</code> file on every server machine in the cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rpc.engine<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.ipc.SecureRpcEngine<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Be advised that if the <code>hbase.security.authentication</code> in the client- and server-side site files do not match, the client will not be able to communicate with the cluster.</p>
</div>
<div class="sect3">
<h4 id="_client_side_configuration_for_simple_user_access_operation_thrift_gateway"><a class="anchor" href="#_client_side_configuration_for_simple_user_access_operation_thrift_gateway"></a>62.4.1. Client-side Configuration for Simple User Access Operation - Thrift Gateway</h4>
<div class="paragraph">
<p>The Thrift gateway user will need access.
For example, to give the Thrift API user, <code>thrift_server</code>, administrative access, a command such as this one will suffice:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">grant</span> <span class="string"><span class="delimiter">'</span><span class="content">thrift_server</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">RWCA</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information about ACLs, please see the <a href="#hbase.accesscontrol.configuration">Access Control Labels (ACLs)</a> section</p>
</div>
<div class="paragraph">
<p>The Thrift gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the Thrift gateway itself.
All client access via the Thrift gateway will use the Thrift gateway&#8217;s credential and have its privilege.</p>
</div>
</div>
<div class="sect3">
<h4 id="_client_side_configuration_for_simple_user_access_operation_rest_gateway"><a class="anchor" href="#_client_side_configuration_for_simple_user_access_operation_rest_gateway"></a>62.4.2. Client-side Configuration for Simple User Access Operation - REST Gateway</h4>
<div class="paragraph">
<p>The REST gateway will authenticate with HBase using the supplied credential.
No authentication will be performed by the REST gateway itself.
All client access via the REST gateway will use the REST gateway&#8217;s credential and have its privilege.</p>
</div>
<div class="paragraph">
<p>The REST gateway user will need access.
For example, to give the REST API user, <code>rest_server</code>, administrative access, a command such as this one will suffice:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">grant</span> <span class="string"><span class="delimiter">'</span><span class="content">rest_server</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">RWCA</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information about ACLs, please see the <a href="#hbase.accesscontrol.configuration">Access Control Labels (ACLs)</a> section</p>
</div>
<div class="paragraph">
<p>It should be possible for clients to authenticate with the HBase cluster through the REST gateway in a pass-through manner via SPNEGO HTTP authentication.
This is future work.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_securing_access_to_hdfs_and_zookeeper"><a class="anchor" href="#_securing_access_to_hdfs_and_zookeeper"></a>63. Securing Access to HDFS and ZooKeeper</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Secure HBase requires secure ZooKeeper and HDFS so that users cannot access and/or modify the metadata and data from under HBase. HBase uses HDFS (or configured file system) to keep its data files as well as write ahead logs (WALs) and other data. HBase uses ZooKeeper to store some metadata for operations (master address, table locks, recovery state, etc).</p>
</div>
<div class="sect2">
<h3 id="_securing_zookeeper_data"><a class="anchor" href="#_securing_zookeeper_data"></a>63.1. Securing ZooKeeper Data</h3>
<div class="paragraph">
<p>ZooKeeper has a pluggable authentication mechanism to enable access from clients using different methods. ZooKeeper even allows authenticated and un-authenticated clients at the same time. The access to znodes can be restricted by providing Access Control Lists (ACLs) per znode. An ACL contains two components, the authentication method and the principal. ACLs are NOT enforced hierarchically. See <a href="https://zookeeper.apache.org/doc/r3.3.6/zookeeperProgrammers.html#sc_ZooKeeperPluggableAuthentication">ZooKeeper Programmers Guide</a> for details.</p>
</div>
<div class="paragraph">
<p>HBase daemons authenticate to ZooKeeper via SASL and kerberos (See <a href="#zk.sasl.auth">SASL Authentication with ZooKeeper</a>). HBase sets up the znode ACLs so that only the HBase user and the configured hbase superuser (<code>hbase.superuser</code>) can access and modify the data. In cases where ZooKeeper is used for service discovery or sharing state with the client, the znodes created by HBase will also allow anyone (regardless of authentication) to read these znodes (clusterId, master address, meta location, etc), but only the HBase user can modify them.</p>
</div>
</div>
<div class="sect2">
<h3 id="_securing_file_system_hdfs_data"><a class="anchor" href="#_securing_file_system_hdfs_data"></a>63.2. Securing File System (HDFS) Data</h3>
<div class="paragraph">
<p>All of the data under management is kept under the root directory in the file system (<code>hbase.rootdir</code>). Access to the data and WAL files in the filesystem should be restricted so that users cannot bypass the HBase layer, and peek at the underlying data files from the file system. HBase assumes the filesystem used (HDFS or other) enforces permissions hierarchically. If sufficient protection from the file system (both authorization and authentication) is not provided, HBase level authorization control (ACLs, visibility labels, etc) is meaningless since the user can always access the data from the file system.</p>
</div>
<div class="paragraph">
<p>HBase enforces the posix-like permissions 700 (<code>rwx------</code>) to its root directory. It means that only the HBase user can read or write the files in FS. The default setting can be changed by configuring <code>hbase.rootdir.perms</code> in hbase-site.xml. A restart of the active master is needed so that it changes the used permissions. For versions before 1.2.0, you can check whether HBASE-13780 is committed, and if not, you can manually set the permissions for the root directory if needed. Using HDFS, the command would be:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">sudo -u hdfs hadoop fs -chmod 700 /hbase</code></pre>
</div>
</div>
<div class="paragraph">
<p>You should change <code>/hbase</code> if you are using a different <code>hbase.rootdir</code>.</p>
</div>
<div class="paragraph">
<p>In secure mode, SecureBulkLoadEndpoint should be configured and used for properly handing of users files created from MR jobs to the HBase daemons and HBase user. The staging directory in the distributed file system used for bulk load (<code>hbase.bulkload.staging.dir</code>, defaults to <code>/tmp/hbase-staging</code>) should have (mode 711, or <code>rwx&#8212;&#8203;x&#8212;&#8203;x</code>) so that users can access the staging directory created under that parent directory, but cannot do any other operation. See <a href="#hbase.secure.bulkload">Secure Bulk Load</a> for how to configure SecureBulkLoadEndPoint.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_securing_access_to_your_data"><a class="anchor" href="#_securing_access_to_your_data"></a>64. Securing Access To Your Data</h2>
<div class="sectionbody">
<div class="paragraph">
<p>After you have configured secure authentication between HBase client and server processes and gateways, you need to consider the security of your data itself.
HBase provides several strategies for securing your data:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Role-based Access Control (RBAC) controls which users or groups can read and write to a given HBase resource or execute a coprocessor endpoint, using the familiar paradigm of roles.</p>
</li>
<li>
<p>Visibility Labels which allow you to label cells and control access to labelled cells, to further restrict who can read or write to certain subsets of your data.
Visibility labels are stored as tags.
See <a href="#hbase.tags">hbase.tags</a> for more information.</p>
</li>
<li>
<p>Transparent encryption of data at rest on the underlying filesystem, both in HFiles and in the WAL.
This protects your data at rest from an attacker who has access to the underlying filesystem, without the need to change the implementation of the client.
It can also protect against data leakage from improperly disposed disks, which can be important for legal and regulatory compliance.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Server-side configuration, administration, and implementation details of each of these features are discussed below, along with any performance trade-offs.
An example security configuration is given at the end, to show these features all used together, as they might be in a real-world scenario.</p>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
All aspects of security in HBase are in active development and evolving rapidly.
Any strategy you employ for security of your data should be thoroughly tested.
In addition, some of these features are still in the experimental stage of development.
To take advantage of many of these features, you must be running HBase 0.98+ and using the HFile v3 file format.
</td>
</tr>
</table>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Protecting Sensitive Files</div>
<div class="paragraph">
<p>Several procedures in this section require you to copy files between cluster nodes.
When copying keys, configuration files, or other files containing sensitive strings, use a secure method, such as <code>ssh</code>, to avoid leaking sensitive data.</p>
</div>
</td>
</tr>
</table>
</div>
<div id="security.data.basic.server.side" class="olist arabic">
<div class="title">Procedure: Basic Server-Side Configuration</div>
<ol class="arabic">
<li>
<p>Enable HFile v3, by setting <code>hfile.format.version</code> to 3 in <em>hbase-site.xml</em>.
This is the default for HBase 1.0 and newer.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hfile.format.version<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</li>
<li>
<p>Enable SASL and Kerberos authentication for RPC and ZooKeeper, as described in <a href="#security.prerequisites">security.prerequisites</a> and <a href="#zk.sasl.auth">SASL Authentication with ZooKeeper</a>.</p>
</li>
</ol>
</div>
<div class="sect2">
<h3 id="hbase.tags"><a class="anchor" href="#hbase.tags"></a>64.1. Tags</h3>
<div class="paragraph">
<p><em class="firstterm">Tags</em> are a feature of HFile v3.
A tag is a piece of metadata which is part of a cell, separate from the key, value, and version.
Tags are an implementation detail which provides a foundation for other security-related features such as cell-level ACLs and visibility labels.
Tags are stored in the HFiles themselves.
It is possible that in the future, tags will be used to implement other HBase features.
You don&#8217;t need to know a lot about tags in order to use the security features they enable.</p>
</div>
<div class="sect3">
<h4 id="_implementation_details"><a class="anchor" href="#_implementation_details"></a>64.1.1. Implementation Details</h4>
<div class="paragraph">
<p>Every cell can have zero or more tags.
Every tag has a type and the actual tag byte array.</p>
</div>
<div class="paragraph">
<p>Just as row keys, column families, qualifiers and values can be encoded (see <a href="#data.block.encoding.types">data.block.encoding.types</a>), tags can also be encoded as well.
You can enable or disable tag encoding at the level of the column family, and it is enabled by default.
Use the <code>HColumnDescriptor#setCompressionTags(boolean compressTags)</code> method to manage encoding settings on a column family.
You also need to enable the DataBlockEncoder for the column family, for encoding of tags to take effect.</p>
</div>
<div class="paragraph">
<p>You can enable compression of each tag in the WAL, if WAL compression is also enabled, by setting the value of <code>hbase.regionserver.wal.tags.enablecompression</code> to <code>true</code> in <em>hbase-site.xml</em>.
Tag compression uses dictionary encoding.</p>
</div>
<div class="paragraph">
<p>Coprocessors that run server-side on RegionServers can perform get and set operations on cell Tags. Tags are stripped out at the RPC layer before the read response is sent back, so clients do not see these tags.
Tag compression is not supported when using WAL encryption.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.accesscontrol.configuration"><a class="anchor" href="#hbase.accesscontrol.configuration"></a>64.2. Access Control Labels (ACLs)</h3>
<div class="sect3">
<h4 id="_how_it_works"><a class="anchor" href="#_how_it_works"></a>64.2.1. How It Works</h4>
<div class="paragraph">
<p>ACLs in HBase are based upon a user&#8217;s membership in or exclusion from groups, and a given group&#8217;s permissions to access a given resource.
ACLs are implemented as a coprocessor called AccessController.</p>
</div>
<div class="paragraph">
<p>HBase does not maintain a private group mapping, but relies on a <em class="firstterm">Hadoop group mapper</em>, which maps between entities in a directory such as LDAP or Active Directory, and HBase users.
Any supported Hadoop group mapper will work.
Users are then granted specific permissions (Read, Write, Execute, Create, Admin) against resources (global, namespaces, tables, cells, or endpoints).</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
With Kerberos and Access Control enabled, client access to HBase is authenticated and user data is private unless access has been explicitly granted.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>HBase has a simpler security model than relational databases, especially in terms of client operations.
No distinction is made between an insert (new record) and update (of existing record), for example, as both collapse down into a Put.</p>
</div>
<div class="sect4">
<h5 id="_understanding_access_levels"><a class="anchor" href="#_understanding_access_levels"></a>Understanding Access Levels</h5>
<div class="paragraph">
<p>HBase access levels are granted independently of each other and allow for different types of operations at a given scope.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>Read (R)</em> - can read data at the given scope</p>
</li>
<li>
<p><em>Write (W)</em> - can write data at the given scope</p>
</li>
<li>
<p><em>Execute (X)</em> - can execute coprocessor endpoints at the given scope</p>
</li>
<li>
<p><em>Create (C)</em> - can create tables or drop tables (even those they did not create) at the given scope</p>
</li>
<li>
<p><em>Admin (A)</em> - can perform cluster operations such as balancing the cluster or assigning regions at the given scope</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The possible scopes are:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>Superuser</em> - superusers can perform any operation available in HBase, to any resource.
The user who runs HBase on your cluster is a superuser, as are any principals assigned to the configuration property <code>hbase.superuser</code> in <em>hbase-site.xml</em> on the HMaster.</p>
</li>
<li>
<p><em>Global</em> - permissions granted at <em>global</em> scope allow the admin to operate on all tables of the cluster.</p>
</li>
<li>
<p><em>Namespace</em> - permissions granted at <em>namespace</em> scope apply to all tables within a given namespace.</p>
</li>
<li>
<p><em>Table</em> - permissions granted at <em>table</em> scope apply to data or metadata within a given table.</p>
</li>
<li>
<p><em>ColumnFamily</em> - permissions granted at <em>ColumnFamily</em> scope apply to cells within that ColumnFamily.</p>
</li>
<li>
<p><em>Cell</em> - permissions granted at <em>cell</em> scope apply to that exact cell coordinate (key, value, timestamp). This allows for policy evolution along with data.</p>
<div class="paragraph">
<p>To change an ACL on a specific cell, write an updated cell with new ACL to the precise coordinates of the original.</p>
</div>
<div class="paragraph">
<p>If you have a multi-versioned schema and want to update ACLs on all visible versions, you need to write new cells for all visible versions.
The application has complete control over policy evolution.</p>
</div>
<div class="paragraph">
<p>The exception to the above rule is <code>append</code> and <code>increment</code> processing.
Appends and increments can carry an ACL in the operation.
If one is included in the operation, then it will be applied to the result of the <code>append</code> or <code>increment</code>.
Otherwise, the ACL of the existing cell you are appending to or incrementing is preserved.</p>
</div>
</li>
</ul>
</div>
<div class="paragraph">
<p>The combination of access levels and scopes creates a matrix of possible access levels that can be granted to a user.
In a production environment, it is useful to think of access levels in terms of what is needed to do a specific job.
The following list describes appropriate access levels for some common types of HBase users.
It is important not to grant more access than is required for a given user to perform their required tasks.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>Superusers</em> - In a production system, only the HBase user should have superuser access.
In a development environment, an administrator may need superuser access in order to quickly control and manage the cluster.
However, this type of administrator should usually be a Global Admin rather than a superuser.</p>
</li>
<li>
<p><em>Global Admins</em> - A global admin can perform tasks and access every table in HBase.
In a typical production environment, an admin should not have Read or Write permissions to data within tables.</p>
</li>
<li>
<p>A global admin with Admin permissions can perform cluster-wide operations on the cluster, such as balancing, assigning or unassigning regions, or calling an explicit major compaction.
This is an operations role.</p>
</li>
<li>
<p>A global admin with Create permissions can create or drop any table within HBase.
This is more of a DBA-type role.</p>
<div class="paragraph">
<p>In a production environment, it is likely that different users will have only one of Admin and Create permissions.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="paragraph">
<p>In the current implementation, a Global Admin with <code>Admin</code> permission can grant himself <code>Read</code> and <code>Write</code> permissions on a table and gain access to that table&#8217;s data.
For this reason, only grant <code>Global Admin</code> permissions to trusted user who actually need them.</p>
</div>
<div class="paragraph">
<p>Also be aware that a <code>Global Admin</code> with <code>Create</code> permission can perform a <code>Put</code> operation on the ACL table, simulating a <code>grant</code> or <code>revoke</code> and circumventing the authorization check for <code>Global Admin</code> permissions.</p>
</div>
<div class="paragraph">
<p>Due to these issues, be cautious with granting <code>Global Admin</code> privileges.</p>
</div>
</td>
</tr>
</table>
</div>
</li>
<li>
<p><em>Namespace Admins</em> - a namespace admin with <code>Create</code> permissions can create or drop tables within that namespace, and take and restore snapshots.
A namespace admin with <code>Admin</code> permissions can perform operations such as splits or major compactions on tables within that namespace.</p>
</li>
<li>
<p><em>Table Admins</em> - A table admin can perform administrative operations only on that table.
A table admin with <code>Create</code> permissions can create snapshots from that table or restore that table from a snapshot.
A table admin with <code>Admin</code> permissions can perform operations such as splits or major compactions on that table.</p>
</li>
<li>
<p><em>Users</em> - Users can read or write data, or both.
Users can also execute coprocessor endpoints, if given <code>Executable</code> permissions.</p>
</li>
</ul>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 9. Real-World Example of Access Levels</caption>
<colgroup>
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
<col style="width: 25%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Job Title</th>
<th class="tableblock halign-left valign-top">Scope</th>
<th class="tableblock halign-left valign-top">Permissions</th>
<th class="tableblock halign-left valign-top">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Senior Administrator</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Global</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access, Create</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Manages the cluster and gives access to Junior Administrators.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Junior Administrator</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Global</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Create</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Creates tables and gives access to Table Administrators.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Table Administrator</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Table</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Maintains a table from an operations point of view.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Data Analyst</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Table</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Read</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Creates reports from HBase data.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Web Application</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Table</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Read, Write</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Puts data into HBase and uses HBase data to perform operations.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<div class="title">ACL Matrix</div>
<p>For more details on how ACLs map to specific HBase operations and tasks, see <a href="#appendix_acl_matrix">appendix acl matrix</a>.</p>
</div>
</div>
<div class="sect4">
<h5 id="_implementation_details_2"><a class="anchor" href="#_implementation_details_2"></a>Implementation Details</h5>
<div class="paragraph">
<p>Cell-level ACLs are implemented using tags (see <a href="#hbase.tags">Tags</a>). In order to use cell-level ACLs, you must be using HFile v3 and HBase 0.98 or newer.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Files created by HBase are owned by the operating system user running the HBase process.
To interact with HBase files, you should use the API or bulk load facility.</p>
</li>
<li>
<p>HBase does not model "roles" internally in HBase.
Instead, group names can be granted permissions.
This allows external modeling of roles via group membership.
Groups are created and manipulated externally to HBase, via the Hadoop group mapping service.</p>
</li>
</ol>
</div>
</div>
<div class="sect4">
<h5 id="_server_side_configuration"><a class="anchor" href="#_server_side_configuration"></a>Server-Side Configuration</h5>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>As a prerequisite, perform the steps in <a href="#security.data.basic.server.side">Procedure: Basic Server-Side Configuration</a>.</p>
</li>
<li>
<p>Install and configure the AccessController coprocessor, by setting the following properties in <em>hbase-site.xml</em>.
These properties take a list of classes.</p>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If you use the AccessController along with the VisibilityController, the AccessController must come first in the list, because with both components active, the VisibilityController will delegate access control on its system tables to the AccessController.
For an example of using both together, see <a href="#security.example.config">Security Configuration Example</a>.
</td>
</tr>
</table>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController, org.apache.hadoop.hbase.security.token.TokenProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.regionserver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.exec.permission.checks<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Optionally, you can enable transport security, by setting <code>hbase.rpc.protection</code> to <code>privacy</code>.
This requires HBase 0.98.4 or newer.</p>
</div>
</li>
<li>
<p>Set up the Hadoop group mapper in the Hadoop namenode&#8217;s <em>core-site.xml</em>.
This is a Hadoop file, not an HBase file.
Customize it to your site&#8217;s needs.
Following is an example.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.security.LdapGroupsMapping<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.url<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>ldap://server<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.bind.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>Administrator@example-ad.local<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.bind.password<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>****<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.base<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>dc=example-ad,dc=local<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.filter.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>(<span class="entity">&amp;amp;</span>(objectClass=user)(sAMAccountName={0}))<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.filter.group<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>(objectClass=group)<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.attr.member<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>member<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.attr.group.name<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>cn<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</li>
<li>
<p>Optionally, enable the early-out evaluation strategy.
Prior to HBase 0.98.0, if a user was not granted access to a column family, or at least a column qualifier, an AccessDeniedException would be thrown.
HBase 0.98.0 removed this exception in order to allow cell-level exceptional grants.
To restore the old behavior in HBase 0.98.0-0.98.6, set <code>hbase.security.access.early_out</code> to <code>true</code> in <em>hbase-site.xml</em>.
In HBase 0.98.6, the default has been returned to <code>true</code>.</p>
</li>
<li>
<p>Distribute your configuration and restart your cluster for changes to take effect.</p>
</li>
<li>
<p>To test your configuration, log into HBase Shell as a given user and use the <code>whoami</code> command to report the groups your user is part of.
In this example, the user is reported as being a member of the <code>services</code> group.</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; whoami
service (auth:KERBEROS)
groups: services</pre>
</div>
</div>
</li>
</ol>
</div>
</div>
<div class="sect4">
<h5 id="_administration"><a class="anchor" href="#_administration"></a>Administration</h5>
<div class="paragraph">
<p>Administration tasks can be performed from HBase Shell or via an API.</p>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">API Examples</div>
<div class="paragraph">
<p>Many of the API examples below are taken from source files <em>hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java</em> and <em>hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java</em>.</p>
</div>
<div class="paragraph">
<p>Neither the examples, nor the source files they are taken from, are part of the public HBase API, and are provided for illustration only.
Refer to the official API for usage instructions.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>User and Group Administration</p>
<div class="paragraph">
<p>Users and groups are maintained external to HBase, in your directory.</p>
</div>
</li>
<li>
<p>Granting Access To A Namespace, Table, Column Family, or Cell</p>
<div class="paragraph">
<p>There are a few different types of syntax for grant statements.
The first, and most familiar, is as follows, with the table and column family being optional:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">grant</span> <span class="string"><span class="delimiter">'</span><span class="content">user</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">RWXCA</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">TABLE</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">CF</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">CQ</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Groups and users are granted access in the same way, but groups are prefixed with an <code>@</code> symbol.
In the same way, tables and namespaces are specified in the same way, but namespaces are prefixed with an <code>@</code> symbol.</p>
</div>
<div class="paragraph">
<p>It is also possible to grant multiple permissions against the same resource in a single statement, as in this example.
The first sub-clause maps users to ACLs and the second sub-clause specifies the resource.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
HBase Shell support for granting and revoking access at the cell level is for testing and verification support, and should not be employed for production use because it won&#8217;t apply the permissions to cells that don&#8217;t exist yet.
The correct way to apply cell level permissions is to do so in the application code when storing the values.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<div class="title">ACL Granularity and Evaluation Order</div>
<p>ACLs are evaluated from least granular to most granular, and when an ACL is reached that grants permission, evaluation stops.
This means that cell ACLs do not override ACLs at less granularity.</p>
</div>
<div class="exampleblock">
<div class="title">Example 14. HBase Shell</div>
<div class="content">
<div class="ulist">
<ul>
<li>
<p>Global:</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant '@admins', 'RWXCA'</pre>
</div>
</div>
</li>
<li>
<p>Namespace:</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant 'service', 'RWXCA', '@test-NS'</pre>
</div>
</div>
</li>
<li>
<p>Table:</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant 'service', 'RWXCA', 'user'</pre>
</div>
</div>
</li>
<li>
<p>Column Family:</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant '@developers', 'RW', 'user', 'i'</pre>
</div>
</div>
</li>
<li>
<p>Column Qualifier:</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant 'service, 'RW', 'user', 'i', 'foo'</pre>
</div>
</div>
</li>
<li>
<p>Cell:</p>
<div class="paragraph">
<p>The syntax for granting cell ACLs uses the following syntax:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>grant &lt;table&gt;, \
{ '&lt;user-or-group&gt;' =&gt; \
'&lt;permissions&gt;', ... }, \
{ &lt;scanner-specification&gt; }</pre>
</div>
</div>
</li>
<li>
<p><em>&lt;user-or-group&gt;</em> is the user or group name, prefixed with <code>@</code> in the case of a group.</p>
</li>
<li>
<p><em>&lt;permissions&gt;</em> is a string containing any or all of "RWXCA", though only R and W are meaningful at cell scope.</p>
</li>
<li>
<p><em>&lt;scanner-specification&gt;</em> is the scanner specification syntax and conventions used by the 'scan' shell command.
For some examples of scanner specifications, issue the following HBase Shell command.</p>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; help "scan"</pre>
</div>
</div>
<div class="paragraph">
<p>If you need to enable cell acl,the hfile.format.version option in hbase-site.xml should be greater than or equal to 3,and the hbase.security.access.early_out option should be set to false.This example grants read access to the 'testuser' user and read/write access to the 'developers' group, on cells in the 'pii' column which match the filter.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; grant 'user', \
{ '@developers' =&gt; 'RW', 'testuser' =&gt; 'R' }, \
{ COLUMNS =&gt; 'pii', FILTER =&gt; "(PrefixFilter ('test'))" }</pre>
</div>
</div>
<div class="paragraph">
<p>The shell will run a scanner with the given criteria, rewrite the found cells with new ACLs, and store them back to their exact coordinates.</p>
</div>
</li>
</ul>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 15. API</div>
<div class="content">
<div class="paragraph">
<p>The following example shows how to grant access at the table level.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> grantOnTable(<span class="directive">final</span> HBaseTestingUtility util, <span class="directive">final</span> <span class="predefined-type">String</span> user,
<span class="directive">final</span> TableName table, <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> family, <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> qualifier,
<span class="directive">final</span> <span class="predefined-type">Permission</span>.Action... actions) <span class="directive">throws</span> <span class="exception">Exception</span> {
SecureTestUtil.updateACLs(util, <span class="keyword">new</span> <span class="predefined-type">Callable</span>&lt;<span class="predefined-type">Void</span>&gt;() {
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="predefined-type">Void</span> call() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(util.getConfiguration())) {
connection.getAdmin().grant(<span class="keyword">new</span> UserPermission(user, <span class="predefined-type">Permission</span>.newBuilder(table)
.withFamily(family).withQualifier(qualifier).withActions(actions).build()),
<span class="predefined-constant">false</span>);
}
<span class="keyword">return</span> <span class="predefined-constant">null</span>;
}
});
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>To grant permissions at the cell level, you can use the <code>Mutation.setACL</code> method:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Mutation.setACL(<span class="predefined-type">String</span> user, <span class="predefined-type">Permission</span> perms)
Mutation.setACL(<span class="predefined-type">Map</span>&lt;<span class="predefined-type">String</span>, <span class="predefined-type">Permission</span>&gt; perms)</code></pre>
</div>
</div>
<div class="paragraph">
<p>Specifically, this example provides read permission to a user called <code>user1</code> on any cells contained in a particular Put operation:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">put.setACL(<span class="error"></span>user1<span class="error"></span>, <span class="keyword">new</span> <span class="predefined-type">Permission</span>(<span class="predefined-type">Permission</span>.Action.READ))</code></pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Revoking Access Control From a Namespace, Table, Column Family, or Cell</p>
<div class="paragraph">
<p>The <code>revoke</code> command and API are twins of the grant command and API, and the syntax is exactly the same.
The only exception is that you cannot revoke permissions at the cell level.
You can only revoke access that has previously been granted, and a <code>revoke</code> statement is not the same thing as explicit denial to a resource.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
HBase Shell support for granting and revoking access is for testing and verification support, and should not be employed for production use because it won&#8217;t apply the permissions to cells that don&#8217;t exist yet.
The correct way to apply cell-level permissions is to do so in the application code when storing the values.
</td>
</tr>
</table>
</div>
<div class="exampleblock">
<div class="title">Example 16. Revoking Access To a Table</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> revokeFromTable(<span class="directive">final</span> HBaseTestingUtility util, <span class="directive">final</span> <span class="predefined-type">String</span> user,
<span class="directive">final</span> TableName table, <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> family, <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> qualifier,
<span class="directive">final</span> <span class="predefined-type">Permission</span>.Action... actions) <span class="directive">throws</span> <span class="exception">Exception</span> {
SecureTestUtil.updateACLs(util, <span class="keyword">new</span> <span class="predefined-type">Callable</span>&lt;<span class="predefined-type">Void</span>&gt;() {
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="predefined-type">Void</span> call() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(util.getConfiguration())) {
connection.getAdmin().revoke(<span class="keyword">new</span> UserPermission(user, <span class="predefined-type">Permission</span>.newBuilder(table)
.withFamily(family).withQualifier(qualifier).withActions(actions).build()));
}
<span class="keyword">return</span> <span class="predefined-constant">null</span>;
}
});
}</code></pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Showing a User&#8217;s Effective Permissions</p>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; user_permission 'user'
hbase&gt; user_permission '.*'
hbase&gt; user_permission JAVA_REGEX</pre>
</div>
</div>
</li>
</ol>
</div>
<div class="exampleblock">
<div class="title">Example 17. API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> verifyAllowed(User user, AccessTestAction action, <span class="type">int</span> count) <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="keyword">try</span> {
<span class="predefined-type">Object</span> obj = user.runAs(action);
<span class="keyword">if</span> (obj != <span class="predefined-constant">null</span> &amp;&amp; obj <span class="keyword">instanceof</span> <span class="predefined-type">List</span>&amp;lt;?&amp;gt;) {
<span class="predefined-type">List</span>&amp;lt;?&amp;gt; results = (<span class="predefined-type">List</span>&amp;lt;?&amp;gt;) obj;
<span class="keyword">if</span> (results != <span class="predefined-constant">null</span> &amp;&amp; results.isEmpty()) {
fail(<span class="string"><span class="delimiter">&quot;</span><span class="content">Empty non null results from action for user '</span><span class="delimiter">&quot;</span></span> <span class="error">`</span> user.getShortName() <span class="error">`</span> <span class="string"><span class="delimiter">&quot;</span><span class="content">'</span><span class="delimiter">&quot;</span></span>);
}
assertEquals(count, results.size());
}
} <span class="keyword">catch</span> (AccessDeniedException ade) {
fail(<span class="string"><span class="delimiter">&quot;</span><span class="content">Expected action to pass for user '</span><span class="delimiter">&quot;</span></span> <span class="error">`</span> user.getShortName() <span class="error">`</span> <span class="string"><span class="delimiter">&quot;</span><span class="content">' but was denied</span><span class="delimiter">&quot;</span></span>);
}
}</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.visibility.labels"><a class="anchor" href="#hbase.visibility.labels"></a>64.3. Visibility Labels</h3>
<div class="paragraph">
<p>Visibility labels control can be used to only permit users or principals associated with a given label to read or access cells with that label.
For instance, you might label a cell <code>top-secret</code>, and only grant access to that label to the <code>managers</code> group.
Visibility labels are implemented using Tags, which are a feature of HFile v3, and allow you to store metadata on a per-cell basis.
A label is a string, and labels can be combined into expressions by using logical operators (&amp;, |, or !), and using parentheses for grouping.
HBase does not do any kind of validation of expressions beyond basic well-formedness.
Visibility labels have no meaning on their own, and may be used to denote sensitivity level, privilege level, or any other arbitrary semantic meaning.</p>
</div>
<div class="paragraph">
<p>If a user&#8217;s labels do not match a cell&#8217;s label or expression, the user is denied access to the cell.</p>
</div>
<div class="paragraph">
<p>In HBase 0.98.6 and newer, UTF-8 encoding is supported for visibility labels and expressions.
When creating labels using the <code>addLabels(conf, labels)</code> method provided by the <code>org.apache.hadoop.hbase.security.visibility.VisibilityClient</code> class and passing labels in Authorizations via Scan or Get, labels can contain UTF-8 characters, as well as the logical operators normally used in visibility labels, with normal Java notations, without needing any escaping method.
However, when you pass a CellVisibility expression via a Mutation, you must enclose the expression with the <code>CellVisibility.quote()</code> method if you use UTF-8 characters or logical operators.
See <code>TestExpressionParser</code> and the source file <em>hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java</em>.</p>
</div>
<div class="paragraph">
<p>A user adds visibility expressions to a cell during a Put operation.
In the default configuration, the user does not need to have access to a label in order to label cells with it.
This behavior is controlled by the configuration option <code>hbase.security.visibility.mutations.checkauths</code>.
If you set this option to <code>true</code>, the labels the user is modifying as part of the mutation must be associated with the user, or the mutation will fail.
Whether a user is authorized to read a labelled cell is determined during a Get or Scan, and results which the user is not allowed to read are filtered out.
This incurs the same I/O penalty as if the results were returned, but reduces load on the network.</p>
</div>
<div class="paragraph">
<p>Visibility labels can also be specified during Delete operations.
For details about visibility labels and Deletes, see <a href="https://issues.apache.org/jira/browse/HBASE-10885">HBASE-10885</a>.</p>
</div>
<div class="paragraph">
<p>The user&#8217;s effective label set is built in the RPC context when a request is first received by the RegionServer.
The way that users are associated with labels is pluggable.
The default plugin passes through labels specified in Authorizations added to the Get or Scan and checks those against the calling user&#8217;s authenticated labels list.
When the client passes labels for which the user is not authenticated, the default plugin drops them.
You can pass a subset of user authenticated labels via the <code>Get#setAuthorizations(Authorizations(String,&#8230;&#8203;))</code> and <code>Scan#setAuthorizations(Authorizations(String,&#8230;&#8203;));</code> methods.</p>
</div>
<div class="paragraph">
<p>Groups can be granted visibility labels the same way as users. Groups are prefixed with an @ symbol. When checking visibility labels of a user, the server will include the visibility labels of the groups of which the user is a member, together with the user&#8217;s own labels.
When the visibility labels are retrieved using API <code>VisibilityClient#getAuths</code> or Shell command <code>get_auths</code> for a user, we will return labels added specifically for that user alone, not the group level labels.</p>
</div>
<div class="paragraph">
<p>Visibility label access checking is performed by the VisibilityController coprocessor.
You can use interface <code>VisibilityLabelService</code> to provide a custom implementation and/or control the way that visibility labels are stored with cells.
See the source file <em>hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithCustomVisLabService.java</em> for one example.</p>
</div>
<div class="paragraph">
<p>Visibility labels can be used in conjunction with ACLs.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The labels have to be explicitly defined before they can be used in visibility labels. See below for an example of how this can be done.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
There is currently no way to determine which labels have been applied to a cell. See <a href="https://issues.apache.org/jira/browse/HBASE-12470">HBASE-12470</a> for details.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Visibility labels are not currently applied for superusers.
</td>
</tr>
</table>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 10. Examples of Visibility Expressions</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Expression</th>
<th class="tableblock halign-left valign-top">Interpretation</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>fulltime</pre></div></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Allow access to users associated with the fulltime label.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>!public</pre></div></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Allow access to users not associated with the public label.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>( secret | topsecret ) &amp; !probationary</pre></div></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Allow access to users associated with either the secret or topsecret label and not associated with the probationary label.</p></td>
</tr>
</tbody>
</table>
<div class="sect3">
<h4 id="_server_side_configuration_2"><a class="anchor" href="#_server_side_configuration_2"></a>64.3.1. Server-Side Configuration</h4>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>As a prerequisite, perform the steps in <a href="#security.data.basic.server.side">Procedure: Basic Server-Side Configuration</a>.</p>
</li>
<li>
<p>Install and configure the VisibilityController coprocessor by setting the following properties in <em>hbase-site.xml</em>.
These properties take a list of class names.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.visibility.VisibilityController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.visibility.VisibilityController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If you use the AccessController and VisibilityController coprocessors together, the AccessController must come first in the list, because with both components active, the VisibilityController will delegate access control on its system tables to the AccessController.
</td>
</tr>
</table>
</div>
</li>
<li>
<p>Adjust Configuration</p>
<div class="paragraph">
<p>By default, users can label cells with any label, including labels they are not associated with, which means that a user can Put data that he cannot read.
For example, a user could label a cell with the (hypothetical) 'topsecret' label even if the user is not associated with that label.
If you only want users to be able to label cells with labels they are associated with, set <code>hbase.security.visibility.mutations.checkauths</code> to <code>true</code>.
In that case, the mutation will fail if it makes use of labels the user is not associated with.</p>
</div>
</li>
<li>
<p>Distribute your configuration and restart your cluster for changes to take effect.</p>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_administration_2"><a class="anchor" href="#_administration_2"></a>64.3.2. Administration</h4>
<div class="paragraph">
<p>Administration tasks can be performed using the HBase Shell or the Java API.
For defining the list of visibility labels and associating labels with users, the HBase Shell is probably simpler.</p>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">API Examples</div>
<div class="paragraph">
<p>Many of the Java API examples in this section are taken from the source file <em>hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java</em>.
Refer to that file or the API documentation for more context.</p>
</div>
<div class="paragraph">
<p>Neither these examples, nor the source file they were taken from, are part of the public HBase API, and are provided for illustration only.
Refer to the official API for usage instructions.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Define the List of Visibility Labels</p>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; add_labels [ 'admin', 'service', 'developer', 'test' ]</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 18. Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> addLabels() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="predefined-type">PrivilegedExceptionAction</span>&lt;VisibilityLabelsResponse&gt; action = <span class="keyword">new</span> <span class="predefined-type">PrivilegedExceptionAction</span>&lt;VisibilityLabelsResponse&gt;() {
<span class="directive">public</span> VisibilityLabelsResponse run() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="predefined-type">String</span><span class="type">[]</span> labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE, COPYRIGHT, ACCENT,
UNICODE_VIS_TAG, UC1, UC2 };
<span class="keyword">try</span> {
VisibilityClient.addLabels(conf, labels);
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> t) {
<span class="keyword">throw</span> <span class="keyword">new</span> <span class="exception">IOException</span>(t);
}
<span class="keyword">return</span> <span class="predefined-constant">null</span>;
}
};
SUPERUSER.runAs(action);
}</code></pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Associate Labels with Users</p>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; set_auths 'service', [ 'service' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_auths 'testuser', [ 'test' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_auths 'qa', [ 'test', 'developer' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_auths '@qagroup', [ 'test' ]</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 19. Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">void</span> testSetAndGetUserAuths() <span class="directive">throws</span> <span class="predefined-type">Throwable</span> {
<span class="directive">final</span> <span class="predefined-type">String</span> user = <span class="string"><span class="delimiter">&quot;</span><span class="content">user1</span><span class="delimiter">&quot;</span></span>;
<span class="predefined-type">PrivilegedExceptionAction</span>&lt;<span class="predefined-type">Void</span>&gt; action = <span class="keyword">new</span> <span class="predefined-type">PrivilegedExceptionAction</span>&lt;<span class="predefined-type">Void</span>&gt;() {
<span class="directive">public</span> <span class="predefined-type">Void</span> run() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="predefined-type">String</span><span class="type">[]</span> auths = { SECRET, CONFIDENTIAL };
<span class="keyword">try</span> {
VisibilityClient.setAuths(conf, auths, user);
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> e) {
}
<span class="keyword">return</span> <span class="predefined-constant">null</span>;
}
...</code></pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Clear Labels From Users</p>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; clear_auths 'service', [ 'service' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; clear_auths 'testuser', [ 'test' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; clear_auths 'qa', [ 'test', 'developer' ]</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; clear_auths '@qagroup', [ 'test', 'developer' ]</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 20. Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
auths = <span class="keyword">new</span> <span class="predefined-type">String</span><span class="type">[]</span> { SECRET, PUBLIC, CONFIDENTIAL };
VisibilityLabelsResponse response = <span class="predefined-constant">null</span>;
<span class="keyword">try</span> {
response = VisibilityClient.clearAuths(conf, auths, user);
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> e) {
fail(<span class="string"><span class="delimiter">&quot;</span><span class="content">Should not have failed</span><span class="delimiter">&quot;</span></span>);
...
}</code></pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Apply a Label or Expression to a Cell</p>
<div class="paragraph">
<p>The label is only applied when data is written.
The label is associated with a given version of the cell.</p>
</div>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; set_visibility 'user', 'admin|service|developer', { COLUMNS =&gt; 'i' }</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_visibility 'user', 'admin|service', { COLUMNS =&gt; 'pii' }</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_visibility 'user', 'test', { COLUMNS =&gt; [ 'i', 'pii' ], FILTER =&gt; "(PrefixFilter ('test'))" }</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
HBase Shell support for applying labels or permissions to cells is for testing and verification support, and should not be employed for production use because it won&#8217;t apply the labels to cells that don&#8217;t exist yet.
The correct way to apply cell level labels is to do so in the application code when storing the values.
</td>
</tr>
</table>
</div>
<div class="exampleblock">
<div class="title">Example 21. Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">static</span> Table createTableAndWriteDataWithLabels(TableName tableName, <span class="predefined-type">String</span>... labelExps)
<span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Table table = NULL;
<span class="keyword">try</span> {
table = TEST_UTIL.createTable(tableName, fam);
<span class="type">int</span> i = <span class="integer">1</span>;
<span class="predefined-type">List</span>&lt;Put&gt; puts = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;Put&gt;();
<span class="keyword">for</span> (<span class="predefined-type">String</span> labelExp : labelExps) {
Put put = <span class="keyword">new</span> Put(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">row</span><span class="delimiter">&quot;</span></span> + i));
put.add(fam, qual, HConstants.LATEST_TIMESTAMP, value);
put.setCellVisibility(<span class="keyword">new</span> CellVisibility(labelExp));
puts.add(put);
i++;
}
table.put(puts);
} <span class="keyword">finally</span> {
<span class="keyword">if</span> (table != <span class="predefined-constant">null</span>) {
table.flushCommits();
}
}</code></pre>
</div>
</div>
</div>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="reading_cells_with_labels"><a class="anchor" href="#reading_cells_with_labels"></a>64.3.3. Reading Cells with Labels</h4>
<div class="paragraph">
<p>When you issue a Scan or Get, HBase uses your default set of authorizations to
filter out cells that you do not have access to. A superuser can set the default
set of authorizations for a given user by using the <code>set_auths</code> HBase Shell command
or the
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityClient.html#setAuths-org.apache.hadoop.hbase.client.Connection-java.lang.String:A-java.lang.String-">VisibilityClient.setAuths()</a> method.</p>
</div>
<div class="paragraph">
<p>You can specify a different authorization during the Scan or Get, by passing the
AUTHORIZATIONS option in HBase Shell, or the
<a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setAuthorizations-org.apache.hadoop.hbase.security.visibility.Authorizations-">Scan.setAuthorizations()</a>
method if you use the API. This authorization will be combined with your default
set as an additional filter. It will further filter your results, rather than
giving you additional authorization.</p>
</div>
<div class="listingblock">
<div class="title">HBase Shell</div>
<div class="content">
<pre>hbase&gt; get_auths 'myUser'
hbase&gt; scan 'table1', AUTHORIZATIONS =&gt; ['private']</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 22. Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
public <span class="predefined-type">Void</span> run() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="predefined-type">String</span><span class="type">[]</span> auths1 = { SECRET, CONFIDENTIAL };
GetAuthsResponse authsResponse = <span class="predefined-constant">null</span>;
<span class="keyword">try</span> {
VisibilityClient.setAuths(conf, auths1, user);
<span class="keyword">try</span> {
authsResponse = VisibilityClient.getAuths(conf, user);
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> e) {
fail(<span class="string"><span class="delimiter">&quot;</span><span class="content">Should not have failed</span><span class="delimiter">&quot;</span></span>);
}
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> e) {
}
<span class="predefined-type">List</span>&lt;<span class="predefined-type">String</span>&gt; authsList = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;<span class="predefined-type">String</span>&gt;();
<span class="keyword">for</span> (ByteString authBS : authsResponse.getAuthList()) {
authsList.add(Bytes.toString(authBS.toByteArray()));
}
assertEquals(<span class="integer">2</span>, authsList.size());
assertTrue(authsList.contains(SECRET));
assertTrue(authsList.contains(CONFIDENTIAL));
<span class="keyword">return</span> <span class="predefined-constant">null</span>;
}
...</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_implementing_your_own_visibility_label_algorithm"><a class="anchor" href="#_implementing_your_own_visibility_label_algorithm"></a>64.3.4. Implementing Your Own Visibility Label Algorithm</h4>
<div class="paragraph">
<p>Interpreting the labels authenticated for a given get/scan request is a pluggable algorithm.</p>
</div>
<div class="paragraph">
<p>You can specify a custom plugin or plugins by using the property <code>hbase.regionserver.scan.visibility.label.generator.class</code>. The output for the first <code>ScanLabelGenerator</code> will be the input for the next one, until the end of the list.</p>
</div>
<div class="paragraph">
<p>The default implementation, which was implemented in <a href="https://issues.apache.org/jira/browse/HBASE-12466">HBASE-12466</a>, loads two plugins, <code>FeedUserAuthScanLabelGenerator</code> and <code>DefinedSetFilterScanLabelGenerator</code>. See <a href="#reading_cells_with_labels">Reading Cells with Labels</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="_replicating_visibility_tags_as_strings"><a class="anchor" href="#_replicating_visibility_tags_as_strings"></a>64.3.5. Replicating Visibility Tags as Strings</h4>
<div class="paragraph">
<p>As mentioned in the above sections, the interface <code>VisibilityLabelService</code> could be used to implement a different way of storing the visibility expressions in the cells. Clusters with replication enabled also must replicate the visibility expressions to the peer cluster. If <code>DefaultVisibilityLabelServiceImpl</code> is used as the implementation for <code>VisibilityLabelService</code>, all the visibility expression are converted to the corresponding expression based on the ordinals for each visibility label stored in the labels table. During replication, visible cells are also replicated with the ordinal-based expression intact. The peer cluster may not have the same <code>labels</code> table with the same ordinal mapping for the visibility labels. In that case, replicating the ordinals makes no sense. It would be better if the replication occurred with the visibility expressions transmitted as strings. To replicate the visibility expression as strings to the peer cluster, create a <code>RegionServerObserver</code> configuration which works based on the implementation of the <code>VisibilityLabelService</code> interface. The configuration below enables replication of visibility expressions to peer clusters as strings. See <a href="https://issues.apache.org/jira/browse/HBASE-11639">HBASE-11639</a> for more details.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.regionserver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.visibility.VisibilityController$VisibilityReplication<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.encryption.server"><a class="anchor" href="#hbase.encryption.server"></a>64.4. Transparent Encryption of Data At Rest</h3>
<div class="paragraph">
<p>HBase provides a mechanism for protecting your data at rest, in HFiles and the WAL, which reside within HDFS or another distributed filesystem.
A two-tier architecture is used for flexible and non-intrusive key rotation.
"Transparent" means that no implementation changes are needed on the client side.
When data is written, it is encrypted.
When it is read, it is decrypted on demand.</p>
</div>
<div class="sect3">
<h4 id="_how_it_works_2"><a class="anchor" href="#_how_it_works_2"></a>64.4.1. How It Works</h4>
<div class="paragraph">
<p>The administrator provisions a master key for the cluster, which is stored in a key provider accessible to every trusted HBase process, including the HMaster, RegionServers, and clients (such as HBase Shell) on administrative workstations.
The default key provider is integrated with the Java KeyStore API and any key management systems with support for it.
Other custom key provider implementations are possible.
The key retrieval mechanism is configured in the <em>hbase-site.xml</em> configuration file.
The master key may be stored on the cluster servers, protected by a secure KeyStore file, or on an external keyserver, or in a hardware security module.
This master key is resolved as needed by HBase processes through the configured key provider.</p>
</div>
<div class="paragraph">
<p>Next, encryption use can be specified in the schema, per column family, by creating or modifying a column descriptor to include two additional attributes: the name of the encryption algorithm to use (currently only "AES" is supported), and optionally, a data key wrapped (encrypted) with the cluster master key.
If a data key is not explicitly configured for a ColumnFamily, HBase will create a random data key per HFile.
This provides an incremental improvement in security over the alternative.
Unless you need to supply an explicit data key, such as in a case where you are generating encrypted HFiles for bulk import with a given data key, only specify the encryption algorithm in the ColumnFamily schema metadata and let HBase create data keys on demand.
Per Column Family keys facilitate low impact incremental key rotation and reduce the scope of any external leak of key material.
The wrapped data key is stored in the ColumnFamily schema metadata, and in each HFile for the Column Family, encrypted with the cluster master key.
After the Column Family is configured for encryption, any new HFiles will be written encrypted.
To ensure encryption of all HFiles, trigger a major compaction after enabling this feature.</p>
</div>
<div class="paragraph">
<p>When the HFile is opened, the data key is extracted from the HFile, decrypted with the cluster master key, and used for decryption of the remainder of the HFile.
The HFile will be unreadable if the master key is not available.
If a remote user somehow acquires access to the HFile data because of some lapse in HDFS permissions, or from inappropriately discarded media, it will not be possible to decrypt either the data key or the file data.</p>
</div>
<div class="paragraph">
<p>It is also possible to encrypt the WAL.
Even though WALs are transient, it is necessary to encrypt the WALEdits to avoid circumventing HFile protections for encrypted column families, in the event that the underlying filesystem is compromised.
When WAL encryption is enabled, all WALs are encrypted, regardless of whether the relevant HFiles are encrypted.</p>
</div>
</div>
<div class="sect3">
<h4 id="_server_side_configuration_3"><a class="anchor" href="#_server_side_configuration_3"></a>64.4.2. Server-Side Configuration</h4>
<div class="paragraph">
<p>This procedure assumes you are using the default Java keystore implementation.
If you are using a custom implementation, check its documentation and adjust accordingly.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Create a secret key of appropriate length for AES encryption, using the
<code>keytool</code> utility.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ keytool -keystore /path/to/hbase/conf/hbase.jks \
-storetype jceks -storepass **** \
-genseckey -keyalg AES -keysize 128 \
-alias &lt;alias&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Replace <em class="replaceable">****</em> with the password for the keystore file and &lt;alias&gt; with the username of the HBase service account, or an arbitrary string.
If you use an arbitrary string, you will need to configure HBase to use it, and that is covered below.
Specify a keysize that is appropriate.
Do not specify a separate password for the key, but press <kbd>Return</kbd> when prompted.</p>
</div>
</li>
<li>
<p>Set appropriate permissions on the keyfile and distribute it to all the HBase
servers.</p>
<div class="paragraph">
<p>The previous command created a file called <em>hbase.jks</em> in the HBase <em>conf/</em> directory.
Set the permissions and ownership on this file such that only the HBase service account user can read the file, and securely distribute the key to all HBase servers.</p>
</div>
</li>
<li>
<p>Configure the HBase daemons.</p>
<div class="paragraph">
<p>Set the following properties in <em>hbase-site.xml</em> on the region servers, to configure HBase daemons to use a key provider backed by the KeyStore file or retrieving the cluster master key.
In the example below, replace <em class="replaceable">****</em> with the password.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.keyprovider<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.keyprovider.parameters<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>jceks:///path/to/hbase/conf/hbase.jks?password=****<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>By default, the HBase service account name will be used to resolve the cluster master key.
However, you can store it with an arbitrary alias (in the <code>keytool</code> command). In that case, set the following property to the alias you used.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.master.key.name<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>my-alias<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>You also need to be sure your HFiles use HFile v3, in order to use transparent encryption.
This is the default configuration for HBase 1.0 onward.
For previous versions, set the following property in your <em>hbase-site.xml</em> file.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hfile.format.version<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Optionally, you can use a different cipher provider, either a Java Cryptography Encryption (JCE) algorithm provider or a custom HBase cipher implementation.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>JCE:</p>
<div class="ulist">
<ul>
<li>
<p>Install a signed JCE provider (supporting <code>AES/CTR/NoPadding</code> mode with 128 bit keys)</p>
</li>
<li>
<p>Add it with highest preference to the JCE site configuration file <em>$JAVA_HOME/lib/security/java.security</em>.</p>
</li>
<li>
<p>Update <code>hbase.crypto.algorithm.aes.provider</code> and <code>hbase.crypto.algorithm.rng.provider</code> options in <em class="path">hbase-site.xml</em>.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Custom HBase Cipher:</p>
<div class="ulist">
<ul>
<li>
<p>Implement <code>org.apache.hadoop.hbase.io.crypto.CipherProvider</code>.</p>
</li>
<li>
<p>Add the implementation to the server classpath.</p>
</li>
<li>
<p>Update <code>hbase.crypto.cipherprovider</code> in <em>hbase-site.xml</em>.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</li>
<li>
<p>Configure WAL encryption.</p>
<div class="paragraph">
<p>Configure WAL encryption in every RegionServer&#8217;s <em>hbase-site.xml</em>, by setting the following properties.
You can include these in the HMaster&#8217;s <em>hbase-site.xml</em> as well, but the HMaster does not have a WAL and will not use them.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.hlog.reader.impl<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.hlog.writer.impl<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.wal.encryption<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</li>
<li>
<p>Configure permissions on the <em>hbase-site.xml</em> file.</p>
<div class="paragraph">
<p>Because the keystore password is stored in the hbase-site.xml, you need to ensure that only the HBase user can read the <em>hbase-site.xml</em> file, using file ownership and permissions.</p>
</div>
</li>
<li>
<p>Restart your cluster.</p>
<div class="paragraph">
<p>Distribute the new configuration file to all nodes and restart your cluster.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_administration_3"><a class="anchor" href="#_administration_3"></a>64.4.3. Administration</h4>
<div class="paragraph">
<p>Administrative tasks can be performed in HBase Shell or the Java API.</p>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">Java API</div>
<div class="paragraph">
<p>Java API examples in this section are taken from the source file <em>hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java</em>.
.</p>
</div>
<div class="paragraph">
<p>Neither these examples, nor the source files they are taken from, are part of the public HBase API, and are provided for illustration only.
Refer to the official API for usage instructions.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Enable Encryption on a Column Family</dt>
<dd>
<p>To enable encryption on a column family, you can either use HBase Shell or the Java API.
After enabling encryption, trigger a major compaction.
When the major compaction completes, the HFiles will be encrypted.</p>
</dd>
<dt class="hdlist1">Rotate the Data Key</dt>
<dd>
<p>To rotate the data key, first change the ColumnFamily key in the column descriptor, then trigger a major compaction.
When compaction is complete, all HFiles will be re-encrypted using the new data key.
Until the compaction completes, the old HFiles will still be readable using the old key.</p>
</dd>
<dt class="hdlist1">Switching Between Using a Random Data Key and Specifying A Key</dt>
<dd>
<p>If you configured a column family to use a specific key and you want to return to the default behavior of using a randomly-generated key for that column family, use the Java API to alter the <code>HColumnDescriptor</code> so that no value is sent with the key <code>ENCRYPTION_KEY</code>.</p>
</dd>
<dt class="hdlist1">Rotate the Master Key</dt>
<dd>
<p>To rotate the master key, first generate and distribute the new key.
Then update the KeyStore to contain a new master key, and keep the old master key in the KeyStore using a different alias.
Next, configure fallback to the old master key in the <em>hbase-site.xml</em> file.</p>
</dd>
<dt class="hdlist1"></dt>
</dl>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.secure.bulkload"><a class="anchor" href="#hbase.secure.bulkload"></a>64.5. Secure Bulk Load</h3>
<div class="paragraph">
<p>Bulk loading in secure mode is a bit more involved than normal setup, since the client has to transfer the ownership of the files generated from the MapReduce job to HBase.
Secure bulk loading is implemented by a coprocessor, named
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.html">SecureBulkLoadEndpoint</a>,
which uses a staging directory configured by the configuration property <code>hbase.bulkload.staging.dir</code>, which defaults to
<em>/tmp/hbase-staging/</em>.</p>
</div>
<div class="ulist">
<div class="title">Secure Bulk Load Algorithm</div>
<ul>
<li>
<p>One time only, create a staging directory which is world-traversable and owned by the user which runs HBase (mode 711, or <code>rwx&#8212;&#8203;x&#8212;&#8203;x</code>). A listing of this directory will look similar to the following:</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ls -ld /tmp/hbase-staging
drwx--x--x 2 hbase hbase 68 3 Sep 14:54 /tmp/hbase-staging</code></pre>
</div>
</div>
</li>
<li>
<p>A user writes out data to a secure output directory owned by that user.
For example, <em>/user/foo/data</em>.</p>
</li>
<li>
<p>Internally, HBase creates a secret staging directory which is globally readable/writable (<code>-rwxrwxrwx, 777</code>). For example, <em>/tmp/hbase-staging/averylongandrandomdirectoryname</em>.
The name and location of this directory is not exposed to the user.
HBase manages creation and deletion of this directory.</p>
</li>
<li>
<p>The user makes the data world-readable and world-writable, moves it into the random staging directory, then calls the <code>SecureBulkLoadClient#bulkLoadHFiles</code> method.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The strength of the security lies in the length and randomness of the secret directory.</p>
</div>
<div class="paragraph">
<p>To enable secure bulk load, add the following properties to <em>hbase-site.xml</em>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.bulkload.staging.dir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/tmp/hbase-staging<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.token.TokenProvider,
org.apache.hadoop.hbase.security.access.AccessController,org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.secure.enable"><a class="anchor" href="#hbase.secure.enable"></a>64.6. Secure Enable</h3>
<div class="paragraph">
<p>After hbase-2.x, the default 'hbase.security.authorization' changed.
Before hbase-2.x, it defaulted to true, in later HBase versions, the
default became false.
So to enable hbase authorization, the following propertie must be configured in <em>hbase-site.xml</em>.
See <a href="https://issues.apache.org/jira/browse/HBASE-19483">HBASE-19483</a>;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="security.example.config"><a class="anchor" href="#security.example.config"></a>65. Security Configuration Example</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This configuration example includes support for HFile v3, ACLs, Visibility Labels, and transparent encryption of data at rest and the WAL.
All options have been discussed separately in the sections above.</p>
</div>
<div class="exampleblock">
<div class="title">Example 23. Example Security Settings in <em>hbase-site.xml</em></div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="comment">&lt;!-- HFile v3 Support --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hfile.format.version<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- HBase Superuser --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.superuser<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hbase,admin<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Coprocessors for ACLs and Visibility Tags --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController,
org.apache.hadoop.hbase.security.visibility.VisibilityController,
org.apache.hadoop.hbase.security.token.TokenProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController,
org.apache.hadoop.hbase.security.visibility.VisibilityController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.regionserver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.access.AccessController<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Executable ACL for Coprocessor Endpoints --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.exec.permission.checks<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Whether a user needs authorization for a visibility tag to set it on a cell --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.visibility.mutations.checkauth<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>false<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Secure RPC Transport --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.rpc.protection<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>privacy<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Transparent Encryption --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.keyprovider<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.keyprovider.parameters<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>jceks:///path/to/hbase/conf/hbase.jks?password=***<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.master.key.name<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- WAL Encryption --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.hlog.reader.impl<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.hlog.writer.impl<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.wal.encryption<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- For key rotation --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.crypto.master.alternate.key.name<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hbase.old<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- Secure Bulk Load --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.bulkload.staging.dir<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/tmp/hbase-staging<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.security.token.TokenProvider,
org.apache.hadoop.hbase.security.access.AccessController,org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 24. Example Group Mapper in Hadoop <em>core-site.xml</em></div>
<div class="content">
<div class="paragraph">
<p>Adjust these settings to suit your environment.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.security.LdapGroupsMapping<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.url<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>ldap://server<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.bind.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>Administrator@example-ad.local<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.bind.password<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>****<span class="tag">&lt;/value&gt;</span> <span class="comment">&lt;!-- Replace with the actual password --&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.base<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>dc=example-ad,dc=local<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.filter.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>(<span class="entity">&amp;amp;</span>(objectClass=user)(sAMAccountName={0}))<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.filter.group<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>(objectClass=group)<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.attr.member<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>member<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hadoop.security.group.mapping.ldap.search.attr.group.name<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>cn<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<h1 id="_architecture" class="sect0"><a class="anchor" href="#_architecture"></a>Architecture</h1>
<div class="sect1">
<h2 id="arch.overview"><a class="anchor" href="#arch.overview"></a>66. Overview</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="arch.overview.nosql"><a class="anchor" href="#arch.overview.nosql"></a>66.1. NoSQL?</h3>
<div class="paragraph">
<p>HBase is a type of "NoSQL" database.
"NoSQL" is a general term meaning that the database isn&#8217;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.
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.</p>
</div>
<div class="paragraph">
<p>However, HBase has many features which supports both linear and modular scaling.
HBase clusters expand by adding RegionServers that are hosted on commodity class servers.
If a cluster expands from 10 to 20 RegionServers, for example, it doubles both in terms of storage and as well as processing capacity.
An RDBMS can scale well, but only up to a point - specifically, the size of a single database
server - and for the best performance requires specialized hardware and storage devices.
HBase features of note are:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Strongly consistent reads/writes: HBase is not an "eventually consistent" DataStore.
This makes it very suitable for tasks such as high-speed counter aggregation.</p>
</li>
<li>
<p>Automatic sharding: HBase tables are distributed on the cluster via regions, and regions are automatically split and re-distributed as your data grows.</p>
</li>
<li>
<p>Automatic RegionServer failover</p>
</li>
<li>
<p>Hadoop/HDFS Integration: HBase supports HDFS out of the box as its distributed file system.</p>
</li>
<li>
<p>MapReduce: HBase supports massively parallelized processing via MapReduce for using HBase as both source and sink.</p>
</li>
<li>
<p>Java Client API: HBase supports an easy to use Java API for programmatic access.</p>
</li>
<li>
<p>Thrift/REST API: HBase also supports Thrift and REST for non-Java front-ends.</p>
</li>
<li>
<p>Block Cache and Bloom Filters: HBase supports a Block Cache and Bloom Filters for high volume query optimization.</p>
</li>
<li>
<p>Operational Management: HBase provides build-in web-pages for operational insight as well as JMX metrics.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="arch.overview.when"><a class="anchor" href="#arch.overview.when"></a>66.2. When Should I Use HBase?</h3>
<div class="paragraph">
<p>HBase isn&#8217;t suitable for every problem.</p>
</div>
<div class="paragraph">
<p>First, make sure you have enough data.
If you have hundreds of millions or billions of rows, then HBase is a good candidate.
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.</p>
</div>
<div class="paragraph">
<p>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.
Consider moving from an RDBMS to HBase as a complete redesign as opposed to a port.</p>
</div>
<div class="paragraph">
<p>Third, make sure you have enough hardware.
Even HDFS doesn&#8217;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.</p>
</div>
<div class="paragraph">
<p>HBase can run quite well stand-alone on a laptop - but this should be considered a development configuration only.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.overview.hbasehdfs"><a class="anchor" href="#arch.overview.hbasehdfs"></a>66.3. What Is The Difference Between HBase and Hadoop/HDFS?</h3>
<div class="paragraph">
<p><a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">HDFS</a> is a distributed file system that is well suited for the storage of large files.
Its documentation states that it is not, however, a general purpose file system, and does not provide fast individual record lookups in files.
HBase, on the other hand, is built on top of HDFS and provides fast record lookups (and updates) for large tables.
This can sometimes be a point of conceptual confusion.
HBase internally puts your data in indexed "StoreFiles" that exist on HDFS for high-speed lookups.
See the <a href="#datamodel">Data Model</a> and the rest of this chapter for more information on how HBase achieves its goals.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="arch.catalog"><a class="anchor" href="#arch.catalog"></a>67. Catalog Tables</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The catalog table <code>hbase:meta</code> exists as an HBase table and is filtered out of the HBase shell&#8217;s <code>list</code> command, but is in fact a table just like any other.</p>
</div>
<div class="sect2">
<h3 id="arch.catalog.meta"><a class="anchor" href="#arch.catalog.meta"></a>67.1. hbase:meta</h3>
<div class="paragraph">
<p>The <code>hbase:meta</code> table (previously called <code>.META.</code>) keeps a list of all regions in the system, and the location of <code>hbase:meta</code> is stored in ZooKeeper.</p>
</div>
<div class="paragraph">
<p>The <code>hbase:meta</code> table structure is as follows:</p>
</div>
<div class="ulist">
<div class="title">Key</div>
<ul>
<li>
<p>Region key of the format (<code>[table],[region start key],[region id]</code>)</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Values</div>
<ul>
<li>
<p><code>info:regioninfo</code> (serialized <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HRegionInfo.html">HRegionInfo</a> instance for this region)</p>
</li>
<li>
<p><code>info:server</code> (server:port of the RegionServer containing this region)</p>
</li>
<li>
<p><code>info:serverstartcode</code> (start-time of the RegionServer process containing this region)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>When a table is in the process of splitting, two other columns will be created, called <code>info:splitA</code> and <code>info:splitB</code>.
These columns represent the two daughter regions.
The values for these columns are also serialized HRegionInfo instances.
After the region has been split, eventually this row will be deleted.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Note on HRegionInfo</div>
<div class="paragraph">
<p>The empty key is used to denote table start and table end.
A region with an empty start key is the first region in a table.
If a region has both an empty start and an empty end key, it is the only region in the table</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>In the (hopefully unlikely) event that programmatic processing of catalog metadata
is required, see the <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/RegionInfo.html#parseFrom-byte:A-">RegionInfo.parseFrom</a> utility.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.catalog.startup"><a class="anchor" href="#arch.catalog.startup"></a>67.2. Startup Sequencing</h3>
<div class="paragraph">
<p>First, the location of <code>hbase:meta</code> is looked up in ZooKeeper.
Next, <code>hbase:meta</code> is updated with server and startcode values.</p>
</div>
<div class="paragraph">
<p>For information on region-RegionServer assignment, see <a href="#regions.arch.assignment">Region-RegionServer Assignment</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="architecture.client"><a class="anchor" href="#architecture.client"></a>68. Client</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The HBase client finds the RegionServers that are serving the particular row range of interest.
It does this by querying the <code>hbase:meta</code> table.
See <a href="#arch.catalog.meta">hbase:meta</a> for details.
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.
This information is cached in the client so that subsequent requests need not go through the lookup process.
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.</p>
</div>
<div class="paragraph">
<p>See <a href="#master.runtime">Runtime Impact</a> for more information about the impact of the Master on HBase Client communication.</p>
</div>
<div class="paragraph">
<p>Administrative functions are done via an instance of <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html">Admin</a></p>
</div>
<div class="sect2">
<h3 id="client.connections"><a class="anchor" href="#client.connections"></a>68.1. Cluster Connections</h3>
<div class="paragraph">
<p>The API changed in HBase 1.0. For connection configuration information, see <a href="#client_dependencies">Client configuration and dependencies connecting to an HBase cluster</a>.</p>
</div>
<div class="sect3">
<h4 id="_api_as_of_hbase_1_0_0"><a class="anchor" href="#_api_as_of_hbase_1_0_0"></a>68.1.1. API as of HBase 1.0.0</h4>
<div class="paragraph">
<p>It&#8217;s been cleaned up and users are returned Interfaces to work against rather than particular types.
In HBase 1.0, obtain a <code>Connection</code> object from <code>ConnectionFactory</code> and thereafter, get from it instances of <code>Table</code>, <code>Admin</code>, and <code>RegionLocator</code> on an as-need basis.
When done, close the obtained instances.
Finally, be sure to cleanup your <code>Connection</code> instance before exiting.
<code>Connections</code> are heavyweight objects but thread-safe so you can create one for your application and keep the instance around.
<code>Table</code>, <code>Admin</code> and <code>RegionLocator</code> instances are lightweight.
Create as you go and then let go as soon as you are done by closing them.
See the <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/package-summary.html">Client Package Javadoc Description</a> for example usage of the new HBase 1.0 API.</p>
</div>
</div>
<div class="sect3">
<h4 id="_api_before_hbase_1_0_0"><a class="anchor" href="#_api_before_hbase_1_0_0"></a>68.1.2. API before HBase 1.0.0</h4>
<div class="paragraph">
<p>Instances of <code>HTable</code> are the way to interact with an HBase cluster earlier than 1.0.0. <em><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html">Table</a> instances are not thread-safe</em>. Only one thread can use an instance of Table at any given time.
When creating Table instances, it is advisable to use the same <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration">HBaseConfiguration</a> instance.
This will ensure sharing of ZooKeeper and socket instances to the RegionServers which is usually what you want.
For example, this is preferred:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HBaseConfiguration conf = HBaseConfiguration.create();
HTable table1 = <span class="keyword">new</span> HTable(conf, <span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);
HTable table2 = <span class="keyword">new</span> HTable(conf, <span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);</code></pre>
</div>
</div>
<div class="paragraph">
<p>as opposed to this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HBaseConfiguration conf1 = HBaseConfiguration.create();
HTable table1 = <span class="keyword">new</span> HTable(conf1, <span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);
HBaseConfiguration conf2 = HBaseConfiguration.create();
HTable table2 = <span class="keyword">new</span> HTable(conf2, <span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);</code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information about how connections are handled in the HBase client, see <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ConnectionFactory.html">ConnectionFactory</a>.</p>
</div>
<div class="sect4">
<h5 id="client.connection.pooling"><a class="anchor" href="#client.connection.pooling"></a>Connection Pooling</h5>
<div class="paragraph">
<p>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 <code>Connection</code>, as shown in the following example:</p>
</div>
<div class="exampleblock">
<div class="title">Example 25. Pre-Creating a <code>Connection</code></div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">// Create a connection to the cluster.</span>
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TableName.valueOf(tablename))) {
<span class="comment">// use table as needed, the table returned is lightweight</span>
}</code></pre>
</div>
</div>
</div>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title"><code>HTablePool</code> is Deprecated</div>
<div class="paragraph">
<p>Previous versions of this guide discussed <code>HTablePool</code>, which was deprecated in HBase 0.94, 0.95, and 0.96, and removed in 0.98.1, by <a href="https://issues.apache.org/jira/browse/HBASE-6580">HBASE-6580</a>, or <code>HConnection</code>, which is deprecated in HBase 1.0 by <code>Connection</code>.
Please use <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Connection.html">Connection</a> instead.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.writebuffer"><a class="anchor" href="#client.writebuffer"></a>68.2. WriteBuffer and Batch Methods</h3>
<div class="paragraph">
<p>In HBase 1.0 and later, <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</a> is deprecated in favor of <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html">Table</a>. <code>Table</code> does not use autoflush. To do buffered writes, use the BufferedMutator class.</p>
</div>
<div class="paragraph">
<p>In HBase 2.0 and later, <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</a> does not use BufferedMutator to execute the <code>Put</code> operation. Refer to <a href="https://issues.apache.org/jira/browse/HBASE-18500">HBASE-18500</a> for more information.</p>
</div>
<div class="paragraph">
<p>For additional information on write durability, review the <a href="/acid-semantics.html">ACID semantics</a> page.</p>
</div>
<div class="paragraph">
<p>For fine-grained control of batching of <code>Put</code>s or <code>Delete</code>s, see the <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch-java.util.List-java.lang.Object:A-">batch</a> methods on Table.</p>
</div>
</div>
<div class="sect2">
<h3 id="async.client"><a class="anchor" href="#async.client"></a>68.3. Asynchronous Client</h3>
<div class="paragraph">
<p>It is a new API introduced in HBase 2.0 which aims to provide the ability to access HBase asynchronously.</p>
</div>
<div class="paragraph">
<p>You can obtain an <code>AsyncConnection</code> from <code>ConnectionFactory</code>, and then get a asynchronous table instance from it to access HBase. When done, close the <code>AsyncConnection</code> instance(usually when your program exits).</p>
</div>
<div class="paragraph">
<p>For the asynchronous table, most methods have the same meaning with the old <code>Table</code> 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.</p>
</div>
<div class="paragraph">
<p>There are several differences for scan:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>There is still a <code>getScanner</code> method which returns a <code>ResultScanner</code>. You can use it in the old way and it works like the old <code>ClientAsyncPrefetchScanner</code>.</p>
</li>
<li>
<p>There is a <code>scanAll</code> 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.</p>
</li>
<li>
<p>The Observer Pattern. There is a scan method which accepts a <code>ScanResultConsumer</code> as a parameter. It will pass the results to the consumer.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Notice that <code>AsyncTable</code> interface is templatized. The template parameter specifies the type of <code>ScanResultConsumerBase</code> used by scans, which means the observer style scan APIs are different. The two types of scan consumers are - <code>ScanResultConsumer</code> and <code>AdvancedScanResultConsumer</code>.</p>
</div>
<div class="paragraph">
<p><code>ScanResultConsumer</code> 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.</p>
</div>
<div class="paragraph">
<p><code>AdvancedScanResultConsumer</code> 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 <code>org.apache.hadoop.hbase.client.example.HttpProxyExample</code> for how to write fully asynchronous code with it.</p>
</div>
</div>
<div class="sect2">
<h3 id="async.admin"><a class="anchor" href="#async.admin"></a>68.4. Asynchronous Admin</h3>
<div class="paragraph">
<p>You can obtain an <code>AsyncConnection</code> from <code>ConnectionFactory</code>, and then get a <code>AsyncAdmin</code> instance from it to access HBase. Notice that there are two <code>getAdmin</code> methods to get a <code>AsyncAdmin</code> instance. One method has one extra thread pool parameter which is used to execute callbacks. It is designed for normal users. Another method doesn&#8217;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.</p>
</div>
<div class="paragraph">
<p>The default <code>getAdmin</code> methods will return a <code>AsyncAdmin</code> instance which use default configs. If you want to customize some configs, you can use <code>getAdminBuilder</code> methods to get a <code>AsyncAdminBuilder</code> for creating <code>AsyncAdmin</code> instance. Users are free to only set the configs they care about to create a new <code>AsyncAdmin</code> instance.</p>
</div>
<div class="paragraph">
<p>For the <code>AsyncAdmin</code> interface, most methods have the same meaning with the old <code>Admin</code> interface, expect that the return value is wrapped with a CompletableFuture usually.</p>
</div>
<div class="paragraph">
<p>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 <code>rollWALWriter</code> method, it only means the rollWALWriter request was sent to the region server and may need some time to finish the <code>rollWALWriter</code> operation.</p>
</div>
<div class="paragraph">
<p>For region name, we only accept <code>byte[]</code> as the parameter type and it may be a full region name or a encoded region name. For server name, we only accept <code>ServerName</code> as the parameter type. For table name, we only accept <code>TableName</code> as the parameter type. For <code>list*</code> operations, we only accept <code>Pattern</code> as the parameter type if you want to do regex matching.</p>
</div>
</div>
<div class="sect2">
<h3 id="client.external"><a class="anchor" href="#client.external"></a>68.5. External Clients</h3>
<div class="paragraph">
<p>Information on non-Java clients and custom protocols is covered in <a href="#external_apis">Apache HBase External APIs</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="client.filter"><a class="anchor" href="#client.filter"></a>69. Client Request Filters</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</a> and <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> instances can be optionally configured with <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html">filters</a> which are applied on the RegionServer.</p>
</div>
<div class="paragraph">
<p>Filters can be confusing because there are many different types, and it is best to approach them by understanding the groups of Filter functionality.</p>
</div>
<div class="sect2">
<h3 id="client.filter.structural"><a class="anchor" href="#client.filter.structural"></a>69.1. Structural</h3>
<div class="paragraph">
<p>Structural Filters contain other Filters.</p>
</div>
<div class="sect3">
<h4 id="client.filter.structural.fl"><a class="anchor" href="#client.filter.structural.fl"></a>69.1.1. FilterList</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FilterList.html">FilterList</a> represents a list of Filters with a relationship of <code>FilterList.Operator.MUST_PASS_ALL</code> or <code>FilterList.Operator.MUST_PASS_ONE</code> between the Filters.
The following example shows an 'or' between two Filters (checking for either 'my value' or 'my other value' on the same attribute).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">FilterList list = <span class="keyword">new</span> FilterList(FilterList.Operator.MUST_PASS_ONE);
SingleColumnValueFilter filter1 = <span class="keyword">new</span> SingleColumnValueFilter(
cf,
column,
CompareOperator.EQUAL,
Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">my value</span><span class="delimiter">&quot;</span></span>)
);
list.add(filter1);
SingleColumnValueFilter filter2 = <span class="keyword">new</span> SingleColumnValueFilter(
cf,
column,
CompareOperator.EQUAL,
Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">my other value</span><span class="delimiter">&quot;</span></span>)
);
list.add(filter2);
scan.setFilter(list);</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.filter.cv"><a class="anchor" href="#client.filter.cv"></a>69.2. Column Value</h3>
<div class="sect3">
<h4 id="client.filter.cv.scvf"><a class="anchor" href="#client.filter.cv.scvf"></a>69.2.1. SingleColumnValueFilter</h4>
<div class="paragraph">
<p>A SingleColumnValueFilter (see:
<a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html" class="bare">https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html</a>)
can be used to test column values for equivalence (<code>CompareOperaor.EQUAL</code>),
inequality (<code>CompareOperaor.NOT_EQUAL</code>), or ranges (e.g., <code>CompareOperaor.GREATER</code>). The following is an
example of testing equivalence of a column to a String value "my value"&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">SingleColumnValueFilter filter = <span class="keyword">new</span> SingleColumnValueFilter(
cf,
column,
CompareOperaor.EQUAL,
Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">my value</span><span class="delimiter">&quot;</span></span>)
);
scan.setFilter(filter);</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.cv.cvf"><a class="anchor" href="#client.filter.cv.cvf"></a>69.2.2. ColumnValueFilter</h4>
<div class="paragraph">
<p>Introduced in HBase-2.0.0 version as a complementation of SingleColumnValueFilter, ColumnValueFilter
gets matched cell only, while SingleColumnValueFilter gets the entire row
(has other columns and values) to which the matched cell belongs. Parameters of constructor of
ColumnValueFilter are the same as SingleColumnValueFilter.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">ColumnValueFilter filter = <span class="keyword">new</span> ColumnValueFilter(
cf,
column,
CompareOperaor.EQUAL,
Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">my value</span><span class="delimiter">&quot;</span></span>)
);
scan.setFilter(filter);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note. For simple query like "equals to a family:qualifier:value", we highly recommend to use the
following way instead of using SingleColumnValueFilter or ColumnValueFilter:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Scan scan = <span class="keyword">new</span> Scan();
scan.addColumn(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">family</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">qualifier</span><span class="delimiter">&quot;</span></span>));
ValueFilter vf = <span class="keyword">new</span> ValueFilter(CompareOperator.EQUAL,
<span class="keyword">new</span> BinaryComparator(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">value</span><span class="delimiter">&quot;</span></span>)));
scan.setFilter(vf);
...</code></pre>
</div>
</div>
<div class="paragraph">
<p>This scan will restrict to the specified column 'family:qualifier', avoiding scan unrelated
families and columns, which has better performance, and <code>ValueFilter</code> is the condition used to do
the value filtering.</p>
</div>
<div class="paragraph">
<p>But if query is much more complicated beyond this book, then please make your good choice case by case.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.filter.cvp"><a class="anchor" href="#client.filter.cvp"></a>69.3. Column Value Comparators</h3>
<div class="paragraph">
<p>There are several Comparator classes in the Filter package that deserve special mention.
These Comparators are used in concert with other Filters, such as <a href="#client.filter.cv.scvf">SingleColumnValueFilter</a>.</p>
</div>
<div class="sect3">
<h4 id="client.filter.cvp.rcs"><a class="anchor" href="#client.filter.cvp.rcs"></a>69.3.1. RegexStringComparator</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RegexStringComparator.html">RegexStringComparator</a> supports regular expressions for value comparisons.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">RegexStringComparator comp = <span class="keyword">new</span> RegexStringComparator(<span class="string"><span class="delimiter">&quot;</span><span class="content">my.</span><span class="delimiter">&quot;</span></span>); <span class="comment">// any value that starts with 'my'</span>
SingleColumnValueFilter filter = <span class="keyword">new</span> SingleColumnValueFilter(
cf,
column,
CompareOperaor.EQUAL,
comp
);
scan.setFilter(filter);</code></pre>
</div>
</div>
<div class="paragraph">
<p>See the Oracle JavaDoc for <a href="http://download.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html">supported RegEx patterns in Java</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.cvp.substringcomparator"><a class="anchor" href="#client.filter.cvp.substringcomparator"></a>69.3.2. SubstringComparator</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SubstringComparator.html">SubstringComparator</a> can be used to determine if a given substring exists in a value.
The comparison is case-insensitive.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">SubstringComparator comp = <span class="keyword">new</span> SubstringComparator(<span class="string"><span class="delimiter">&quot;</span><span class="content">y val</span><span class="delimiter">&quot;</span></span>); <span class="comment">// looking for 'my value'</span>
SingleColumnValueFilter filter = <span class="keyword">new</span> SingleColumnValueFilter(
cf,
column,
CompareOperaor.EQUAL,
comp
);
scan.setFilter(filter);</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.cvp.bfp"><a class="anchor" href="#client.filter.cvp.bfp"></a>69.3.3. BinaryPrefixComparator</h4>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.html">BinaryPrefixComparator</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.cvp.bc"><a class="anchor" href="#client.filter.cvp.bc"></a>69.3.4. BinaryComparator</h4>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryComparator.html">BinaryComparator</a>.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.filter.kvm"><a class="anchor" href="#client.filter.kvm"></a>69.4. KeyValue Metadata</h3>
<div class="paragraph">
<p>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.</p>
</div>
<div class="sect3">
<h4 id="client.filter.kvm.ff"><a class="anchor" href="#client.filter.kvm.ff"></a>69.4.1. FamilyFilter</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FamilyFilter.html">FamilyFilter</a> can be used to filter on the ColumnFamily.
It is generally a better idea to select ColumnFamilies in the Scan than to do it with a Filter.</p>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.kvm.qf"><a class="anchor" href="#client.filter.kvm.qf"></a>69.4.2. QualifierFilter</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/QualifierFilter.html">QualifierFilter</a> can be used to filter based on Column (aka Qualifier) name.</p>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.kvm.cpf"><a class="anchor" href="#client.filter.kvm.cpf"></a>69.4.3. ColumnPrefixFilter</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.html">ColumnPrefixFilter</a> can be used to filter based on the lead portion of Column (aka Qualifier) names.</p>
</div>
<div class="paragraph">
<p>A ColumnPrefixFilter seeks ahead to the first column matching the prefix in each row and for each involved column family.
It can be used to efficiently get a subset of the columns in very wide rows.</p>
</div>
<div class="paragraph">
<p>Note: The same column qualifier can be used in different column families.
This filter returns all matching columns.</p>
</div>
<div class="paragraph">
<p>Example: Find all columns in a row and family that start with "abc"</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Table t = ...;
<span class="type">byte</span><span class="type">[]</span> row = ...;
<span class="type">byte</span><span class="type">[]</span> family = ...;
<span class="type">byte</span><span class="type">[]</span> prefix = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">abc</span><span class="delimiter">&quot;</span></span>);
Scan scan = <span class="keyword">new</span> Scan(row, row); <span class="comment">// (optional) limit to one row</span>
scan.addFamily(family); <span class="comment">// (optional) limit to one family</span>
<span class="predefined-type">Filter</span> f = <span class="keyword">new</span> ColumnPrefixFilter(prefix);
scan.setFilter(f);
scan.setBatch(<span class="integer">10</span>); <span class="comment">// set this if there could be many columns returned</span>
ResultScanner rs = t.getScanner(scan);
<span class="keyword">for</span> (<span class="predefined-type">Result</span> r = rs.next(); r != <span class="predefined-constant">null</span>; r = rs.next()) {
<span class="keyword">for</span> (Cell cell : result.listCells()) {
<span class="comment">// each cell represents a column</span>
}
}
rs.close();</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.kvm.mcpf"><a class="anchor" href="#client.filter.kvm.mcpf"></a>69.4.4. MultipleColumnPrefixFilter</h4>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.html">MultipleColumnPrefixFilter</a> behaves like ColumnPrefixFilter but allows specifying multiple prefixes.</p>
</div>
<div class="paragraph">
<p>Like ColumnPrefixFilter, MultipleColumnPrefixFilter efficiently seeks ahead to the first column matching the lowest prefix and also seeks past ranges of columns between prefixes.
It can be used to efficiently get discontinuous sets of columns from very wide rows.</p>
</div>
<div class="paragraph">
<p>Example: Find all columns in a row and family that start with "abc" or "xyz"</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Table t = ...;
<span class="type">byte</span><span class="type">[]</span> row = ...;
<span class="type">byte</span><span class="type">[]</span> family = ...;
<span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> prefixes = <span class="keyword">new</span> <span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> {Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">abc</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">xyz</span><span class="delimiter">&quot;</span></span>)};
Scan scan = <span class="keyword">new</span> Scan(row, row); <span class="comment">// (optional) limit to one row</span>
scan.addFamily(family); <span class="comment">// (optional) limit to one family</span>
<span class="predefined-type">Filter</span> f = <span class="keyword">new</span> MultipleColumnPrefixFilter(prefixes);
scan.setFilter(f);
scan.setBatch(<span class="integer">10</span>); <span class="comment">// set this if there could be many columns returned</span>
ResultScanner rs = t.getScanner(scan);
<span class="keyword">for</span> (<span class="predefined-type">Result</span> r = rs.next(); r != <span class="predefined-constant">null</span>; r = rs.next()) {
<span class="keyword">for</span> (Cell cell : result.listCells()) {
<span class="comment">// each cell represents a column</span>
}
}
rs.close();</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="client.filter.kvm.crf"><a class="anchor" href="#client.filter.kvm.crf"></a>69.4.5. ColumnRangeFilter</h4>
<div class="paragraph">
<p>A <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnRangeFilter.html">ColumnRangeFilter</a> allows efficient intra row scanning.</p>
</div>
<div class="paragraph">
<p>A ColumnRangeFilter can seek ahead to the first matching column for each involved column family.
It can be used to efficiently get a 'slice' of the columns of a very wide row.
i.e.
you have a million columns in a row but you only want to look at columns bbbb-bbdd.</p>
</div>
<div class="paragraph">
<p>Note: The same column qualifier can be used in different column families.
This filter returns all matching columns.</p>
</div>
<div class="paragraph">
<p>Example: Find all columns in a row and family between "bbbb" (inclusive) and "bbdd" (inclusive)</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Table t = ...;
<span class="type">byte</span><span class="type">[]</span> row = ...;
<span class="type">byte</span><span class="type">[]</span> family = ...;
<span class="type">byte</span><span class="type">[]</span> startColumn = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">bbbb</span><span class="delimiter">&quot;</span></span>);
<span class="type">byte</span><span class="type">[]</span> endColumn = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">bbdd</span><span class="delimiter">&quot;</span></span>);
Scan scan = <span class="keyword">new</span> Scan(row, row); <span class="comment">// (optional) limit to one row</span>
scan.addFamily(family); <span class="comment">// (optional) limit to one family</span>
<span class="predefined-type">Filter</span> f = <span class="keyword">new</span> ColumnRangeFilter(startColumn, <span class="predefined-constant">true</span>, endColumn, <span class="predefined-constant">true</span>);
scan.setFilter(f);
scan.setBatch(<span class="integer">10</span>); <span class="comment">// set this if there could be many columns returned</span>
ResultScanner rs = t.getScanner(scan);
<span class="keyword">for</span> (<span class="predefined-type">Result</span> r = rs.next(); r != <span class="predefined-constant">null</span>; r = rs.next()) {
<span class="keyword">for</span> (Cell cell : result.listCells()) {
<span class="comment">// each cell represents a column</span>
}
}
rs.close();</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note: Introduced in HBase 0.92</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.filter.row"><a class="anchor" href="#client.filter.row"></a>69.5. RowKey</h3>
<div class="sect3">
<h4 id="client.filter.row.rf"><a class="anchor" href="#client.filter.row.rf"></a>69.5.1. RowFilter</h4>
<div class="paragraph">
<p>It is generally a better idea to use the startRow/stopRow methods on Scan for row selection, however <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RowFilter.html">RowFilter</a> can also be used.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="client.filter.utility"><a class="anchor" href="#client.filter.utility"></a>69.6. Utility</h3>
<div class="sect3">
<h4 id="client.filter.utility.fkof"><a class="anchor" href="#client.filter.utility.fkof"></a>69.6.1. FirstKeyOnlyFilter</h4>
<div class="paragraph">
<p>This is primarily used for rowcount jobs.
See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html">FirstKeyOnlyFilter</a>.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="architecture.master"><a class="anchor" href="#architecture.master"></a>70. Master</h2>
<div class="sectionbody">
<div class="paragraph">
<p><code>HMaster</code> is the implementation of the Master Server.
The Master server is responsible for monitoring all RegionServer instances in the cluster, and is the interface for all metadata changes.
In a distributed cluster, the Master typically runs on the <a href="#arch.hdfs.nn">NameNode</a>.
J Mohamed Zahoor goes into some more detail on the Master Architecture in this blog posting, <a href="http://blog.zahoor.in/2012/08/hbase-hmaster-architecture/">HBase HMaster Architecture </a>.</p>
</div>
<div class="sect2">
<h3 id="master.startup"><a class="anchor" href="#master.startup"></a>70.1. Startup Behavior</h3>
<div class="paragraph">
<p>If run in a multi-Master environment, all Masters compete to run the cluster.
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.</p>
</div>
</div>
<div class="sect2">
<h3 id="master.runtime"><a class="anchor" href="#master.runtime"></a>70.2. Runtime Impact</h3>
<div class="paragraph">
<p>A common dist-list question involves what happens to an HBase cluster when the Master goes down.
Because the HBase client talks directly to the RegionServers, the cluster can still function in a "steady state". Additionally, per <a href="#arch.catalog">Catalog Tables</a>, <code>hbase:meta</code> exists as an HBase table and is not resident in the Master.
However, the Master controls critical functions such as RegionServer failover and completing region splits.
So while the cluster can still run for a short time without the Master, the Master should be restarted as soon as possible.</p>
</div>
</div>
<div class="sect2">
<h3 id="master.api"><a class="anchor" href="#master.api"></a>70.3. Interface</h3>
<div class="paragraph">
<p>The methods exposed by <code>HMasterInterface</code> are primarily metadata-oriented methods:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Table (createTable, modifyTable, removeTable, enable, disable)</p>
</li>
<li>
<p>ColumnFamily (addColumn, modifyColumn, removeColumn)</p>
</li>
<li>
<p>Region (move, assign, unassign) For example, when the <code>Admin</code> method <code>disableTable</code> is invoked, it is serviced by the Master server.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="master.processes"><a class="anchor" href="#master.processes"></a>70.4. Processes</h3>
<div class="paragraph">
<p>The Master runs several background threads:</p>
</div>
<div class="sect3">
<h4 id="master.processes.loadbalancer"><a class="anchor" href="#master.processes.loadbalancer"></a>70.4.1. LoadBalancer</h4>
<div class="paragraph">
<p>Periodically, and when there are no regions in transition, a load balancer will run and move regions around to balance the cluster&#8217;s load.
See <a href="#balancer_config">Balancer</a> for configuring this property.</p>
</div>
<div class="paragraph">
<p>See <a href="#regions.arch.assignment">Region-RegionServer Assignment</a> for more information on region assignment.</p>
</div>
</div>
<div class="sect3">
<h4 id="master.processes.catalog"><a class="anchor" href="#master.processes.catalog"></a>70.4.2. CatalogJanitor</h4>
<div class="paragraph">
<p>Periodically checks and cleans up the <code>hbase:meta</code> table.
See <a href="#arch.catalog.meta">hbase:meta</a> for more information on the meta table.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="master.wal"><a class="anchor" href="#master.wal"></a>70.5. MasterProcWAL</h3>
<div class="paragraph">
<p>HMaster records administrative operations and their running states, such as the handling of a crashed server,
table creation, and other DDLs, into its own WAL file. The WALs are stored under the MasterProcWALs
directory. The Master WALs are not like RegionServer WALs. Keeping up the Master WAL allows
us run a state machine that is resilient across Master failures. For example, if a HMaster was in the
middle of creating a table encounters an issue and fails, the next active HMaster can take up where
the previous left off and carry the operation to completion. Since hbase-2.0.0, a
new AssignmentManager (A.K.A AMv2) was introduced and the HMaster handles region assignment
operations, server crash processing, balancing, etc., all via AMv2 persisting all state and
transitions into MasterProcWALs rather than up into ZooKeeper, as we do in hbase-1.x.</p>
</div>
<div class="paragraph">
<p>See <a href="#amv2">AMv2 Description for Devs</a> (and <a href="#pv2">Procedure Framework (Pv2): <a href="https://issues.apache.org/jira/browse/HBASE-12439">HBASE-12439</a></a> for its basis) if you would like to learn more about the new
AssignmentManager.</p>
</div>
<div class="sect3">
<h4 id="master.wal.conf"><a class="anchor" href="#master.wal.conf"></a>70.5.1. Configurations for MasterProcWAL</h4>
<div class="paragraph">
<p>Here are the list of configurations that effect MasterProcWAL operation.
You should not have to change your defaults.</p>
</div>
<div id="hbase.procedure.store.wal.periodic.roll.msec" class="dlist">
<dl>
<dt class="hdlist1"><strong><code>hbase.procedure.store.wal.periodic.roll.msec</code></strong></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>Frequency of generating a new WAL</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>1h (3600000 in msec)</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.store.wal.roll.threshold" class="dlist">
<dl>
<dt class="hdlist1"><strong><code>hbase.procedure.store.wal.roll.threshold</code></strong></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>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.</p>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>32MB (33554432 in byte)</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.store.wal.warn.threshold" class="dlist">
<dl>
<dt class="hdlist1"><strong><code>hbase.procedure.store.wal.warn.threshold</code></strong></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>If the number of WALs goes beyond this threshold, the following message should appear in the HMaster log with WARN level when rolling.</p>
</div>
<div class="literalblock">
<div class="content">
<pre>procedure WALs count=xx above the warning threshold 64. check running procedures to see if something is stuck.</pre>
</div>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>64</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.store.wal.max.retries.before.roll" class="dlist">
<dl>
<dt class="hdlist1"><strong><code>hbase.procedure.store.wal.max.retries.before.roll</code></strong></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>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.</p>
</div>
<div class="literalblock">
<div class="content">
<pre>unable to sync slots, retry=xx</pre>
</div>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
<div id="hbase.procedure.store.wal.sync.failure.roll.max" class="dlist">
<dl>
<dt class="hdlist1"><strong><code>hbase.procedure.store.wal.sync.failure.roll.max</code></strong></dt>
<dd>
<div class="paragraph">
<div class="title">Description</div>
<p>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.</p>
</div>
<div class="literalblock">
<div class="content">
<pre>Sync slots after log roll failed, abort.</pre>
</div>
</div>
<div class="paragraph">
<div class="title">Default</div>
<p><code>3</code></p>
</div>
</dd>
</dl>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="regionserver.arch"><a class="anchor" href="#regionserver.arch"></a>71. RegionServer</h2>
<div class="sectionbody">
<div class="paragraph">
<p><code>HRegionServer</code> is the RegionServer implementation.
It is responsible for serving and managing regions.
In a distributed cluster, a RegionServer runs on a <a href="#arch.hdfs.dn">DataNode</a>.</p>
</div>
<div class="sect2">
<h3 id="regionserver.arch.api"><a class="anchor" href="#regionserver.arch.api"></a>71.1. Interface</h3>
<div class="paragraph">
<p>The methods exposed by <code>HRegionRegionInterface</code> contain both data-oriented and region-maintenance methods:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Data (get, put, delete, next, etc.)</p>
</li>
<li>
<p>Region (splitRegion, compactRegion, etc.) For example, when the <code>Admin</code> method <code>majorCompact</code> 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.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="regionserver.arch.processes"><a class="anchor" href="#regionserver.arch.processes"></a>71.2. Processes</h3>
<div class="paragraph">
<p>The RegionServer runs a variety of background threads:</p>
</div>
<div class="sect3">
<h4 id="regionserver.arch.processes.compactsplit"><a class="anchor" href="#regionserver.arch.processes.compactsplit"></a>71.2.1. CompactSplitThread</h4>
<div class="paragraph">
<p>Checks for splits and handle minor compactions.</p>
</div>
</div>
<div class="sect3">
<h4 id="regionserver.arch.processes.majorcompact"><a class="anchor" href="#regionserver.arch.processes.majorcompact"></a>71.2.2. MajorCompactionChecker</h4>
<div class="paragraph">
<p>Checks for major compactions.</p>
</div>
</div>
<div class="sect3">
<h4 id="regionserver.arch.processes.memstore"><a class="anchor" href="#regionserver.arch.processes.memstore"></a>71.2.3. MemStoreFlusher</h4>
<div class="paragraph">
<p>Periodically flushes in-memory writes in the MemStore to StoreFiles.</p>
</div>
</div>
<div class="sect3">
<h4 id="regionserver.arch.processes.log"><a class="anchor" href="#regionserver.arch.processes.log"></a>71.2.4. LogRoller</h4>
<div class="paragraph">
<p>Periodically checks the RegionServer&#8217;s WAL.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_coprocessors"><a class="anchor" href="#_coprocessors"></a>71.3. Coprocessors</h3>
<div class="paragraph">
<p>Coprocessors were added in 0.92.
There is a thorough <a href="https://blogs.apache.org/hbase/entry/coprocessor_introduction">Blog Overview of CoProcessors</a> posted.
Documentation will eventually move to this reference guide, but the blog is the most current information available at this time.</p>
</div>
</div>
<div class="sect2">
<h3 id="block.cache"><a class="anchor" href="#block.cache"></a>71.4. Block Cache</h3>
<div class="paragraph">
<p>HBase provides two different BlockCache implementations to cache data read from HDFS:
the default on-heap <code>LruBlockCache</code> and the <code>BucketCache</code>, which is (usually) off-heap.
This section discusses benefits and drawbacks of each implementation, how to choose the
appropriate option, and configuration options for each.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Block Cache Reporting: UI</div>
<div class="paragraph">
<p>See the RegionServer UI for detail on caching deploy.
See configurations, sizings, current usage, time-in-the-cache, and even detail on block counts and types.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="_cache_choices"><a class="anchor" href="#_cache_choices"></a>71.4.1. Cache Choices</h4>
<div class="paragraph">
<p><code>LruBlockCache</code> is the original implementation, and is entirely within the Java heap.
<code>BucketCache</code> is optional and mainly intended for keeping block cache data off-heap, although <code>BucketCache</code> can also be a file-backed cache.
In file-backed we can either use it in the file mode or the mmaped mode.
We also have pmem mode where the bucket cache resides on the persistent memory device.</p>
</div>
<div class="paragraph">
<p>When you enable BucketCache, you are enabling a two tier caching system. We used to describe the
tiers as "L1" and "L2" but have deprecated this terminology as of hbase-2.0.0. The "L1" cache referred to an
instance of LruBlockCache and "L2" to an off-heap BucketCache. Instead, when BucketCache is enabled,
all DATA blocks are kept in the BucketCache tier and meta blocks&#8201;&#8212;&#8201;INDEX and BLOOM blocks&#8201;&#8212;&#8201;are on-heap in the <code>LruBlockCache</code>.
Management of these two tiers and the policy that dictates how blocks move between them is done by <code>CombinedBlockCache</code>.</p>
</div>
</div>
<div class="sect3">
<h4 id="cache.configurations"><a class="anchor" href="#cache.configurations"></a>71.4.2. General Cache Configurations</h4>
<div class="paragraph">
<p>Apart from the cache implementation itself, you can set some general configuration options to control how the cache performs.
See <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html">CacheConfig</a>.
After setting any of these options, restart or rolling restart your cluster for the configuration to take effect.
Check logs for errors or unexpected behavior.</p>
</div>
<div class="paragraph">
<p>See also <a href="#blockcache.prefetch">Prefetch Option for Blockcache</a>, which discusses a new option introduced in <a href="https://issues.apache.org/jira/browse/HBASE-9857">HBASE-9857</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="block.cache.design"><a class="anchor" href="#block.cache.design"></a>71.4.3. LruBlockCache Design</h4>
<div class="paragraph">
<p>The LruBlockCache is an LRU cache that contains three levels of block priority to allow for scan-resistance and in-memory ColumnFamilies:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>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.
The advantage is that scanned blocks are more likely to get evicted than blocks that are getting more usage.</p>
</li>
<li>
<p>Multi access priority: If a block in the previous priority group is accessed again, it upgrades to this priority.
It is thus part of the second group considered during evictions.</p>
</li>
<li>
<p>In-memory access priority: If the block&#8217;s family was configured to be "in-memory", it will be part of this priority disregarding the number of times it was accessed.
Catalog tables are configured like this.
This group is the last one considered during evictions.</p>
<div class="paragraph">
<p>To mark a column family as in-memory, call</p>
</div>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HColumnDescriptor.setInMemory(<span class="predefined-constant">true</span>);</code></pre>
</div>
</div>
<div class="paragraph">
<p>if creating a table from java, or set <code>IN_MEMORY &#8658; true</code> when creating or altering a table in the shell: e.g.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">003</span>:<span class="integer">0</span>&gt; create <span class="string"><span class="delimiter">'</span><span class="content">t</span><span class="delimiter">'</span></span>, {NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">f</span><span class="delimiter">'</span></span>, IN_MEMORY =&gt; <span class="string"><span class="delimiter">'</span><span class="content">true</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information, see the LruBlockCache source</p>
</div>
</div>
<div class="sect3">
<h4 id="block.cache.usage"><a class="anchor" href="#block.cache.usage"></a>71.4.4. LruBlockCache Usage</h4>
<div class="paragraph">
<p>Block caching is enabled by default for all the user tables which means that any read operation will load the LRU cache.
This might be good for a large number of use cases, but further tunings are usually required in order to achieve better performance.
An important concept is the <a href="http://en.wikipedia.org/wiki/Working_set_size">working set size</a>, 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&#8217;s needed to answer the queries over a short amount of time.</p>
</div>
<div class="paragraph">
<p>The way to calculate how much memory is available in HBase for caching is:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">number of region servers * heap size * hfile.block.cache.size * <span class="float">0.99</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The default value for the block cache is 0.4 which represents 40% of the available heap.
The last value (99%) is the default acceptable loading factor in the LRU cache after which eviction is started.
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.
Here are some examples:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>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.</p>
</li>
<li>
<p>20 region servers with the heap size set to 8 GB and a default block cache size will have 63.3 of block cache.</p>
</li>
<li>
<p>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.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Your data is not the only resident of the block cache.
Here are others that you may have to take into account:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Catalog Tables</dt>
<dd>
<p>The <code>hbase:meta</code> table is forced into the block cache and have the in-memory priority which means that they are harder to evict.</p>
</dd>
</dl>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The hbase:meta tables can occupy a few MBs depending on the number of regions.
</td>
</tr>
</table>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">HFiles Indexes</dt>
<dd>
<p>An <em>HFile</em> is the file format that HBase uses to store data in HDFS.
It contains a multi-layered index which allows HBase to seek to the data without having to read the whole file.
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.
For big data sets it&#8217;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&#8217;t used.</p>
</dd>
<dt class="hdlist1">Keys</dt>
<dd>
<p>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 <a href="#keysize">Try to minimize row and column sizes</a>.</p>
</dd>
<dt class="hdlist1">Bloom Filters</dt>
<dd>
<p>Just like the HFile indexes, those data structures (when enabled) are stored in the LRU.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>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.
For keys, sampling can be done by using the HFile command line tool and look for the average key size metric.
Since HBase 0.98.3, you can view details on BlockCache stats and metrics in a special Block Cache section in the UI.</p>
</div>
<div class="paragraph">
<p>It&#8217;s generally bad to use block caching when the WSS doesn&#8217;t fit in memory.
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.
One of the reasons is that the churn generated by the evictions will trigger more garbage collections unnecessarily.
Here are two use cases:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>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.
Setting block caching on such a table is a waste of memory and CPU cycles, more so that it will generate more garbage to pick up by the JVM.
For more information on monitoring GC, see <a href="#trouble.log.gc">JVM Garbage Collection Logs</a>.</p>
</li>
<li>
<p>Mapping a table: In a typical MapReduce job that takes a table in input, every row will be read only once so there&#8217;s no need to put them into the block cache.
The Scan object has the option of turning this off via the setCaching method (set it to false). You can still keep block caching turned on on this table if you need fast random read access.
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&#8217;s currently in use.</p>
</li>
</ul>
</div>
<div class="sect4">
<h5 id="data.blocks.in.fscache"><a class="anchor" href="#data.blocks.in.fscache"></a>Caching META blocks only (DATA blocks in fscache)</h5>
<div class="paragraph">
<p>An interesting setup is one where we cache META blocks only and we read DATA blocks in on each access.
If the DATA blocks fit inside fscache, this alternative may make sense when access is completely random across a very large dataset.
To enable this setup, alter your table and for each column family set <code>BLOCKCACHE &#8658; 'false'</code>.
You are 'disabling' the BlockCache for this column family only. You can never disable the caching of META blocks.
Since <a href="https://issues.apache.org/jira/browse/HBASE-4683">HBASE-4683 Always cache index and bloom blocks</a>, we will cache META blocks even if the BlockCache is disabled.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="offheap.blockcache"><a class="anchor" href="#offheap.blockcache"></a>71.4.5. Off-heap Block Cache</h4>
<div class="sect4">
<h5 id="enable.bucketcache"><a class="anchor" href="#enable.bucketcache"></a>How to Enable BucketCache</h5>
<div class="paragraph">
<p>The usual deploy of BucketCache is via a managing class that sets up two caching tiers:
an on-heap cache implemented by LruBlockCache and a second cache implemented with BucketCache.
The managing class is <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.html">CombinedBlockCache</a> by default.
The previous link describes the caching 'policy' implemented by CombinedBlockCache.
In short, it works by keeping meta blocks&#8201;&#8212;&#8201;INDEX and BLOOM in the on-heap LruBlockCache tier&#8201;&#8212;&#8201;and DATA blocks are kept in the BucketCache tier.</p>
</div>
<div class="exampleblock">
<div class="content">
<div class="dlist">
<dl>
<dt class="hdlist1">Pre-hbase-2.0.0 versions</dt>
<dd>
<p>Fetching will always be slower when fetching from BucketCache in pre-hbase-2.0.0,
as compared to the native on-heap LruBlockCache. However, latencies tend to be less
erratic across time, because there is less garbage collection when you use BucketCache since it is managing BlockCache allocations, not the GC.
If the BucketCache is deployed in off-heap mode, this memory is not managed by the GC at all.
This is why you&#8217;d use BucketCache in pre-2.0.0, so your latencies are less erratic,
to mitigate GCs and heap fragmentation, and so you can safely use more memory.
See Nick Dimiduk&#8217;s <a href="http://www.n10k.com/blog/blockcache-101/">BlockCache 101</a> for comparisons running on-heap vs off-heap tests.
Also see <a href="https://people.apache.org/~stack/bc/">Comparing BlockCache Deploys</a> 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.</p>
<div class="paragraph">
<p>In pre-2.0.0,
one can configure the BucketCache so it receives the <code>victim</code> of an LruBlockCache eviction.
All Data and index blocks are cached in L1 first. When eviction happens from L1, the blocks (or <code>victims</code>) will get moved to L2.
Set <code>cacheDataInL1</code> via <code>(HColumnDescriptor.setCacheDataInL1(true)</code> or in the shell, creating or amending column families setting <code>CACHE_DATA_IN_L1</code> to true: e.g.</p>
</div>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">003</span>:<span class="integer">0</span>&gt; create <span class="string"><span class="delimiter">'</span><span class="content">t</span><span class="delimiter">'</span></span>, {NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">t</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {CACHE_DATA_IN_L1 =&gt; <span class="string"><span class="delimiter">'</span><span class="content">true</span><span class="delimiter">'</span></span>}}</code></pre>
</div>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">hbase-2.0.0+ versions</dt>
<dd>
<p>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.
See <a href="#regionserver.offheap.readpath">Offheap read-path</a>. In hbase-2.0.0, off-heap latencies approach those of on-heap cache latencies with the added
benefit of NOT provoking GC.</p>
<div class="paragraph">
<p>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. <code>cacheDataInL1</code> support hase been removed.</p>
</div>
</dd>
</dl>
</div>
</div>
</div>
<div class="paragraph">
<p>The BucketCache Block Cache can be deployed <em>off-heap</em>, <em>file</em> or <em>mmaped</em> file mode.</p>
</div>
<div class="paragraph">
<p>You set which via the <code>hbase.bucketcache.ioengine</code> setting.
Setting it to <code>offheap</code> will have BucketCache make its allocations off-heap, and an ioengine setting of <code>file:PATH_TO_FILE</code> 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 <code>files:PATH_TO_FILE1,PATH_TO_FILE2,PATH_TO_FILE3</code>. BucketCache can be configured to use an mmapped file also. Configure ioengine as <code>mmap:PATH_TO_FILE</code> for this.</p>
</div>
<div class="paragraph">
<p>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.
For such a setup, set <code>hbase.bucketcache.combinedcache.enabled</code> to <code>false</code>.
In this mode, on eviction from L1, blocks go to L2.
When a block is cached, it is cached first in L1.
When we go to look for a cached block, we look first in L1 and if none found, then search L2.
Let us call this deploy format, <em>Raw L1+L2</em>.
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.</p>
</div>
<div class="paragraph">
<p>Other BucketCache configs include: specifying a location to persist cache to across restarts, how many threads to use writing the cache, etc.
See the <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html">CacheConfig.html</a> class for configuration options and descriptions.</p>
</div>
<div class="paragraph">
<p>To check it enabled, look for the log line describing cache setup; it will detail how BucketCache has been deployed.
Also see the UI. It will detail the cache tiering and their configuration.</p>
</div>
<div class="sect5">
<h6 id="_bucketcache_example_configuration"><a class="anchor" href="#_bucketcache_example_configuration"></a>BucketCache Example Configuration</h6>
<div class="paragraph">
<p>This sample provides a configuration for a 4 GB off-heap BucketCache with a 1 GB on-heap cache.</p>
</div>
<div class="paragraph">
<p>Configuration is performed on the RegionServer.</p>
</div>
<div class="paragraph">
<p>Setting <code>hbase.bucketcache.ioengine</code> and <code>hbase.bucketcache.size</code> &gt; 0 enables <code>CombinedBlockCache</code>.
Let us presume that the RegionServer has been set to run with a 5G heap: i.e. <code>HBASE_HEAPSIZE=5g</code>.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>First, edit the RegionServer&#8217;s <em>hbase-env.sh</em> and set <code>HBASE_OFFHEAPSIZE</code> to a value greater than the off-heap size wanted, in this case, 4 GB (expressed as 4G). Let&#8217;s set it to 5G.
That&#8217;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.
DFSClient in RegionServer can make use of off-heap memory). See <a href="#direct.memory">Direct Memory Usage In HBase</a>.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HBASE_OFFHEAPSIZE=<span class="integer">5</span>G</code></pre>
</div>
</div>
</li>
<li>
<p>Next, add the following configuration to the RegionServer&#8217;s <em>hbase-site.xml</em>.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.bucketcache.ioengine<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>offheap<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hfile.block.cache.size<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>0.2<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.bucketcache.size<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>4196<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</li>
<li>
<p>Restart or rolling restart your cluster, and check the logs for any issues.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>In the above, we set the BucketCache to be 4G.
We configured the on-heap LruBlockCache have 20% (0.2) of the RegionServer&#8217;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).</p>
</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-10641">HBASE-10641</a> introduced the ability to configure multiple sizes for the buckets of the BucketCache, in HBase 0.98 and newer.
To configurable multiple bucket sizes, configure the new property <code>hbase.bucketcache.bucket.sizes</code> to a comma-separated list of block sizes, ordered from smallest to largest, with no spaces.
The goal is to optimize the bucket sizes based on your data access patterns.
The following example configures buckets of size 4096 and 8192.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.bucketcache.bucket.sizes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>4096,8192<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div id="direct.memory" class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Direct Memory Usage In HBase</div>
<div class="paragraph">
<p>The default maximum direct memory varies by JVM.
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 <a href="#perf.hdfs.configs.localread">Leveraging local data</a>), 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 * <code>hbase.dfs.client.read.shortcircuit.buffer.size</code> where <code>hbase.dfs.client.read.shortcircuit.buffer.size</code> is set to 128k in HBase&#8201;&#8212;&#8201;see <em>hbase-default.xml</em> default configurations.
If you do off-heap block caching, you&#8217;ll be making use of direct memory.
The RPCServer uses a ByteBuffer pool. From 2.0.0, these buffers are off-heap ByteBuffers.
Starting your JVM, make sure the <code>-XX:MaxDirectMemorySize</code> setting in <em>conf/hbase-env.sh</em> considers off-heap BlockCache (<code>hbase.bucketcache.size</code>), 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.
The value allocated by <code>MaxDirectMemorySize</code> must not exceed physical RAM, and is likely to be less than the total available RAM due to other memory requirements and system constraints.</p>
</div>
<div class="paragraph">
<p>You can see how much memory&#8201;&#8212;&#8201;on-heap and off-heap/direct&#8201;&#8212;&#8201;a RegionServer is configured to use and how much it is using at any one time by looking at the <em>Server Metrics: Memory</em> tab in the UI.
It can also be gotten via JMX.
In particular the direct memory currently used by the server can be found on the <code>java.nio.type=BufferPool,name=direct</code> bean.
Terracotta has a <a href="http://terracotta.org/documentation/4.0/bigmemorygo/configuration/storage-options">good write up</a> on using off-heap memory in Java.
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.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">hbase.bucketcache.percentage.in.combinedcache</div>
<div class="paragraph">
<p>This is a pre-HBase 1.0 configuration removed because it was confusing.
It was a float that you would set to some value between 0.0 and 1.0.
Its default was 0.9.
If the deploy was using CombinedBlockCache, then the LruBlockCache L1 size was calculated to be <code>(1 - hbase.bucketcache.percentage.in.combinedcache) * size-of-bucketcache</code> and the BucketCache size was <code>hbase.bucketcache.percentage.in.combinedcache * size-of-bucket-cache</code>.
where size-of-bucket-cache itself is EITHER the value of the configuration <code>hbase.bucketcache.size</code> IF it was specified as Megabytes OR <code>hbase.bucketcache.size</code> * <code>-XX:MaxDirectMemorySize</code> if <code>hbase.bucketcache.size</code> is between 0 and 1.0.</p>
</div>
<div class="paragraph">
<p>In 1.0, it should be more straight-forward.
Onheap LruBlockCache size is set as a fraction of java heap using <code>hfile.block.cache.size setting</code> (not the best name) and BucketCache is set as above in absolute Megabytes.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_compressed_blockcache"><a class="anchor" href="#_compressed_blockcache"></a>71.4.6. Compressed BlockCache</h4>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-11331">HBASE-11331</a> introduced lazy BlockCache decompression, more simply referred to as compressed BlockCache.
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.</p>
</div>
<div class="paragraph">
<p>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.
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.</p>
</div>
<div class="paragraph">
<p>The compressed BlockCache is disabled by default. To enable it, set <code>hbase.block.data.cachecompressed</code> to <code>true</code> in <em>hbase-site.xml</em> on all RegionServers.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="regionserver_splitting_implementation"><a class="anchor" href="#regionserver_splitting_implementation"></a>71.5. RegionServer Splitting Implementation</h3>
<div class="paragraph">
<p>As write requests are handled by the region server, they accumulate in an in-memory storage system called the <em>memstore</em>. Once the memstore fills, its content are written to disk as additional store files. This event is called a <em>memstore flush</em>. As store files accumulate, the RegionServer will <a href="#compaction">compact</a> 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.</p>
</div>
<div class="paragraph">
<p>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&#8217;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 <em>daughter regions</em> do not rewrite all the data into new files immediately. Instead, they create small files similar to symbolic link files, named <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/Reference.html">Reference files</a>, 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.</p>
</div>
<div class="paragraph">
<p>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 <code>.META.</code> 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 <a href="#regionserver_split_process_image">RegionServer Split Process</a>. 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.</p>
</div>
<div id="regionserver_split_process_image" class="imageblock">
<div class="content">
<img src="images/region_split_process.png" alt="Region Split Process">
</div>
<div class="title">Figure 1. RegionServer Split Process</div>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The RegionServer decides locally to split the region, and prepares the split. <strong>THE SPLIT TRANSACTION IS STARTED.</strong> 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 <code>/hbase/region-in-transition/region-name</code>, and sets the znode&#8217;s state to <code>SPLITTING</code>.</p>
</li>
<li>
<p>The Master learns about this znode, since it has a watcher for the parent <code>region-in-transition</code> znode.</p>
</li>
<li>
<p>The RegionServer creates a sub-directory named <code>.splits</code> under the parent’s <code>region</code> directory in HDFS.</p>
</li>
<li>
<p>The RegionServer closes the parent region and marks the region as offline in its local data structures. <strong>THE SPLITTING REGION IS NOW OFFLINE.</strong> At this point, client requests coming to the parent region will throw <code>NotServingRegionException</code>. The client will retry with some backoff. The closing region is flushed.</p>
</li>
<li>
<p>The RegionServer creates region directories under the <code>.splits</code> directory, for daughter
regions A and B, and creates necessary data structures. Then it splits the store files,
in the sense that it creates two Reference files per store file in the parent region.
Those reference files will point to the parent region&#8217;s files.</p>
</li>
<li>
<p>The RegionServer creates the actual region directory in HDFS, and moves the reference files for each daughter.</p>
</li>
<li>
<p>The RegionServer sends a <code>Put</code> request to the <code>.META.</code> table, to set the parent as offline in the <code>.META.</code> table and add information about daughter regions. At this point, there won’t be individual entries in <code>.META.</code> for the daughters. Clients will see that the parent region is split if they scan <code>.META.</code>, but won’t know about the daughters until they appear in <code>.META.</code>. Also, if this <code>Put</code> to <code>.META</code>. 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 <code>.META.</code> update, though, the region split will be rolled-forward by Master.</p>
</li>
<li>
<p>The RegionServer opens daughters A and B in parallel.</p>
</li>
<li>
<p>The RegionServer adds the daughters A and B to <code>.META.</code>, together with information that it hosts the regions. <strong>THE SPLIT REGIONS (DAUGHTERS WITH REFERENCES TO PARENT) ARE NOW ONLINE.</strong> After this point, clients can discover the new regions and issue requests to them. Clients cache the <code>.META.</code> entries locally, but when they make requests to the RegionServer or <code>.META.</code>, their caches will be invalidated, and they will learn about the new regions from <code>.META.</code>.</p>
</li>
<li>
<p>The RegionServer updates znode <code>/hbase/region-in-transition/region-name</code> in ZooKeeper to state <code>SPLIT</code>, so that the master can learn about it. The balancer can freely re-assign the daughter regions to other region servers if necessary. <strong>THE SPLIT TRANSACTION IS NOW FINISHED.</strong></p>
</li>
<li>
<p>After the split, <code>.META.</code> 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&#8217;s files. If not, the parent region will be removed.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="wal"><a class="anchor" href="#wal"></a>71.6. Write Ahead Log (WAL)</h3>
<div class="sect3">
<h4 id="purpose.wal"><a class="anchor" href="#purpose.wal"></a>71.6.1. Purpose</h4>
<div class="paragraph">
<p>The <em>Write Ahead Log (WAL)</em> records all changes to data in HBase, to file-based storage.
Under normal operations, the WAL is not needed because data changes move from the MemStore to StoreFiles.
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.
If writing to the WAL fails, the entire operation to modify the data fails.</p>
</div>
<div class="paragraph">
<p>HBase uses an implementation of the <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/wal/WAL.html">WAL</a> interface.
Usually, there is only one instance of a WAL per RegionServer. An exception
is the RegionServer that is carrying <em>hbase:meta</em>; the <em>meta</em> table gets its
own dedicated WAL.
The RegionServer records Puts and Deletes to its WAL, before recording them
these Mutations <a href="#store.memstore">MemStore</a> for the affected <a href="#store">Store</a>.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">The HLog</div>
<div class="paragraph">
<p>Prior to 2.0, the interface for WALs in HBase was named <code>HLog</code>.
In 0.94, HLog was the name of the implementation of the WAL.
You will likely find references to the HLog in documentation tailored to these older versions.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The WAL resides in HDFS in the <em>/hbase/WALs/</em> directory, with subdirectories per region.</p>
</div>
<div class="paragraph">
<p>For more general information about the concept of write ahead logs, see the Wikipedia
<a href="http://en.wikipedia.org/wiki/Write-ahead_logging">Write-Ahead Log</a> article.</p>
</div>
</div>
<div class="sect3">
<h4 id="wal.providers"><a class="anchor" href="#wal.providers"></a>71.6.2. WAL Providers</h4>
<div class="paragraph">
<p>In HBase, there are a number of WAL imlementations (or 'Providers'). Each is known
by a short name label (that unfortunately is not always descriptive). You set the provider in
<em>hbase-site.xml</em> passing the WAL provder short-name as the value on the
<em>hbase.wal.provider</em> property (Set the provider for <em>hbase:meta</em> using the
<em>hbase.wal.meta_provider</em> property, otherwise it uses the same provider configured
by <em>hbase.wal.provider</em>).</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>asyncfs</em>: The <strong>default</strong>. New since hbase-2.0.0 (HBASE-15536, HBASE-14790). This <em>AsyncFSWAL</em> 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 <a href="https://www.slideshare.net/HBaseCon/apache-hbase-improvements-and-practices-at-xiaomi">Apache HBase Improements and Practices at Xiaomi</a> at slide 14 onward for more detail on implementation.</p>
</li>
<li>
<p><em>filesystem</em>: This was the default in hbase-1.x releases. It is built on the blocking <em>DFSClient</em> and writes to replicas in classic <em>DFSCLient</em> pipeline mode. In logs it identifies as <em>FSHLog</em> or <em>FSHLogProvider</em>.</p>
</li>
<li>
<p><em>multiwal</em>: This provider is made of multiple instances of <em>asyncfs</em> or <em>filesystem</em>. See the next section for more on <em>multiwal</em>.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Look for the lines like the below in the RegionServer log to see which provider is in place (The below shows the default AsyncFSWALProvider):</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-04-02 13:22:37,983 INFO [regionserver/ve0528:16020] wal.WALFactory: Instantiating WALProvider of type class org.apache.hadoop.hbase.wal.AsyncFSWALProvider</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
As the <em>AsyncFSWAL</em> 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 <em>asyncfs</em>, if failed, we will fall back to use <em>filesystem</em>. And notice that this may not always work, so if you still have problem starting HBase due to the problem of starting <em>AsyncFSWAL</em>, please specify <em>filesystem</em> explicitly in the config file.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
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 <em>FileSystem</em>, 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.
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_multiwal"><a class="anchor" href="#_multiwal"></a>71.6.3. MultiWAL</h4>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>HBase 1.0 introduces support MultiWal in <a href="https://issues.apache.org/jira/browse/HBASE-5699">HBASE-5699</a>. 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.</p>
</div>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<div class="title">Configure MultiWAL</div>
<p>To configure MultiWAL for a RegionServer, set the value of the property <code>hbase.wal.provider</code> to <code>multiwal</code> by pasting in the following XML:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.wal.provider<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>multiwal<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Restart the RegionServer for the changes to take effect.</p>
</div>
<div class="paragraph">
<p>To disable MultiWAL for a RegionServer, unset the property and restart the RegionServer.</p>
</div>
</div>
<div class="sect3">
<h4 id="wal_flush"><a class="anchor" href="#wal_flush"></a>71.6.4. WAL Flushing</h4>
<div class="paragraph">
<p>TODO (describe).</p>
</div>
</div>
<div class="sect3">
<h4 id="_wal_splitting"><a class="anchor" href="#_wal_splitting"></a>71.6.5. WAL Splitting</h4>
<div class="paragraph">
<p>A RegionServer serves many regions.
All of the regions in a region server share the same active WAL file.
Each edit in the WAL file includes information about which region it belongs to.
When a region is opened, the edits in the WAL file which belong to that region need to be replayed.
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.
The process of grouping the WAL edits by region is called <em>log splitting</em>.
It is a critical process for recovering data if a region server fails.</p>
</div>
<div class="paragraph">
<p>Log splitting is done by the HMaster during cluster start-up or by the ServerShutdownHandler as a region server shuts down.
So that consistency is guaranteed, affected regions are unavailable until data is restored.
All WAL edits need to be recovered and replayed before a given region can become available again.
As a result, regions affected by log splitting are unavailable until the process completes.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Log Splitting, Step by Step</div>
<ol class="arabic">
<li>
<p>The <em>/hbase/WALs/&lt;host&gt;,&lt;port&gt;,&lt;startcode&gt;</em> directory is renamed.</p>
<div class="paragraph">
<p>Renaming the directory is important because a RegionServer may still be up and accepting requests even if the HMaster thinks it is down.
If the RegionServer does not respond immediately and does not heartbeat its ZooKeeper session, the HMaster may interpret this as a RegionServer failure.
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.</p>
</div>
<div class="paragraph">
<p>The new directory is named according to the following pattern:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/WALs/&lt;host&gt;,&lt;port&gt;,&lt;startcode&gt;-splitting</pre>
</div>
</div>
<div class="paragraph">
<p>An example of such a renamed directory might look like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/WALs/srv.example.com,60020,1254173957298-splitting</pre>
</div>
</div>
</li>
<li>
<p>Each log file is split, one at a time.</p>
<div class="paragraph">
<p>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&#8217;s region.
At the same time, the splitter starts several writer threads.
Writer threads pick up a corresponding buffer and write the edit entries in the buffer to a temporary recovered edit file.
The temporary edit file is stored to disk with the following naming pattern:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/&lt;table_name&gt;/&lt;region_id&gt;/recovered.edits/.temp</pre>
</div>
</div>
<div class="paragraph">
<p>This file is used to store all the edits in the WAL log for this region.
After log splitting completes, the <em>.temp</em> file is renamed to the sequence ID of the first log written to the file.</p>
</div>
<div class="paragraph">
<p>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.
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.</p>
</div>
</li>
<li>
<p>After log splitting is complete, each affected region is assigned to a RegionServer.</p>
<div class="paragraph">
<p>When the region is opened, the <em>recovered.edits</em> folder is checked for recovered edits files.
If any such files are present, they are replayed by reading the edits and saving them to the MemStore.
After all edit files are replayed, the contents of the MemStore are written to disk (HFile) and the edit files are deleted.</p>
</div>
</li>
</ol>
</div>
<div class="sect4">
<h5 id="_handling_of_errors_during_log_splitting"><a class="anchor" href="#_handling_of_errors_during_log_splitting"></a>Handling of Errors During Log Splitting</h5>
<div class="paragraph">
<p>If you set the <code>hbase.hlog.split.skip.errors</code> option to <code>true</code>, errors are treated as follows:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Any error encountered during splitting will be logged.</p>
</li>
<li>
<p>The problematic WAL log will be moved into the <em>.corrupt</em> directory under the hbase <code>rootdir</code>,</p>
</li>
<li>
<p>Processing of the WAL will continue</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>If the <code>hbase.hlog.split.skip.errors</code> option is set to <code>false</code>, the default, the exception will be propagated and the split will be logged as failed.
See <a href="https://issues.apache.org/jira/browse/HBASE-2958">HBASE-2958 When
hbase.hlog.split.skip.errors is set to false, we fail the split but that&#8217;s it</a>.
We need to do more than just fail split if this flag is set.</p>
</div>
<div class="sect5">
<h6 id="_how_eofexceptions_are_treated_when_splitting_a_crashed_regionserver_s_wals"><a class="anchor" href="#_how_eofexceptions_are_treated_when_splitting_a_crashed_regionserver_s_wals"></a>How EOFExceptions are treated when splitting a crashed RegionServer&#8217;s WALs</h6>
<div class="paragraph">
<p>If an EOFException occurs while splitting logs, the split proceeds even when <code>hbase.hlog.split.skip.errors</code> is set to <code>false</code>.
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.
For background, see <a href="https://issues.apache.org/jira/browse/HBASE-2643">HBASE-2643 Figure how to deal with eof splitting logs</a></p>
</div>
</div>
</div>
<div class="sect4">
<h5 id="_performance_improvements_during_log_splitting"><a class="anchor" href="#_performance_improvements_during_log_splitting"></a>Performance Improvements during Log Splitting</h5>
<div class="paragraph">
<p>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. <a href="#distributed.log.splitting">Enabling or Disabling Distributed Log Splitting</a> was developed to improve performance during log splitting.</p>
</div>
<div id="distributed.log.splitting" class="paragraph">
<div class="title">Enabling or Disabling Distributed Log Splitting</div>
<p>Distributed log processing is enabled by default since HBase 0.92.
The setting is controlled by the <code>hbase.master.distributed.log.splitting</code> property, which can be set to <code>true</code> or <code>false</code>, but defaults to <code>true</code>.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_wal_splitting_based_on_procedurev2"><a class="anchor" href="#_wal_splitting_based_on_procedurev2"></a>71.6.6. WAL splitting based on procedureV2</h4>
<div class="paragraph">
<p>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.</p>
</div>
<div id="background" class="paragraph">
<div class="title">Background</div>
<p>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.</p>
</div>
<div id="implementation.on.master.side" class="paragraph">
<div class="title">Implementation on Master side</div>
<p>During ServerCrashProcedure, SplitWALManager will create one SplitWALProcedure for each WAL file which should be split. Then each SplitWALProcedure will spawn a SplitWalRemoteProcedure to send the request to region server.
SplitWALProcedure is a StateMachineProcedure and here is the state transfer diagram.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/WAL_splitting.png" alt="WAL splitting">
</div>
<div class="title">Figure 2. WAL_splitting_coordination</div>
</div>
<div id="implementation.on.region.server.side" class="paragraph">
<div class="title">Implementation on Region Server side</div>
<p>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.</p>
</div>
<div id="preformance" class="paragraph">
<div class="title">Performance</div>
<p>According to tests on a cluster which has 5 regionserver and 1 master.
procedureV2 coordinated WAL splitting has a better performance than ZK coordinated WAL splitting no master when restarting the whole cluster or one region server crashing.</p>
</div>
<div id="enable.this.feature" class="paragraph">
<div class="title">Enable this feature</div>
<p>To enable this feature, first we should ensure our package of HBase already contains these code. If not, please upgrade the package of HBase cluster without any configuration change first.
Then change configuration 'hbase.split.wal.zk.coordinated' to false. Rolling upgrade the master with new configuration. Now WAL splitting are handled by our new implementation.
But region server are still trying to grab tasks from zookeeper, we can rolling upgrade the region servers with the new configuration to stop that.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>steps as follows:</p>
<div class="ulist">
<ul>
<li>
<p>Upgrade whole cluster to get the new Implementation.</p>
</li>
<li>
<p>Upgrade Master with new configuration 'hbase.split.wal.zk.coordinated'=false.</p>
</li>
<li>
<p>Upgrade region server to stop grab tasks from zookeeper.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="wal.compression"><a class="anchor" href="#wal.compression"></a>71.6.7. WAL Compression</h4>
<div class="paragraph">
<p>The content of the WAL can be compressed using LRU Dictionary compression.
This can be used to speed up WAL replication to different datanodes.
The dictionary can store up to 2<sup>15</sup> elements; eviction starts after this number is exceeded.</p>
</div>
<div class="paragraph">
<p>To enable WAL compression, set the <code>hbase.regionserver.wal.enablecompression</code> property to <code>true</code>.
The default value for this property is <code>false</code>.
By default, WAL tag compression is turned on when WAL compression is enabled.
You can turn off WAL tag compression by setting the <code>hbase.regionserver.wal.tags.enablecompression</code> property to 'false'.</p>
</div>
<div class="paragraph">
<p>A possible downside to WAL compression is that we lose more data from the last block in the WAL if it ill-terminated
mid-write. If entries in this last block were added with new dictionary entries but we failed persist the amended
dictionary because of an abrupt termination, a read of this last block may not be able to resolve last-written entries.</p>
</div>
</div>
<div class="sect3">
<h4 id="wal.durability"><a class="anchor" href="#wal.durability"></a>71.6.8. Durability</h4>
<div class="paragraph">
<p>It is possible to set <em>durability</em> on each Mutation or on a Table basis. Options include:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>SKIP_WAL</em>: Do not write Mutations to the WAL (See the next section, <a href="#wal.disable">Disabling the WAL</a>).</p>
</li>
<li>
<p><em>ASYNC_WAL</em>: 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.</p>
</li>
<li>
<p><em>SYNC_WAL</em>: The <strong>default</strong>. Each edit is sync&#8217;d to HDFS before we return success to the client.</p>
</li>
<li>
<p><em>FSYNC_WAL</em>: Each edit is fsync&#8217;d to HDFS and the filesystem before we return success to the client.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Do not confuse the <em>ASYNC_WAL</em> option on a Mutation or Table with the <em>AsyncFSWAL</em> writer; they are distinct
options unfortunately closely named</p>
</div>
</div>
<div class="sect3">
<h4 id="arch.custom.wal.dir"><a class="anchor" href="#arch.custom.wal.dir"></a>71.6.9. Custom WAL Directory</h4>
<div class="paragraph">
<p>HBASE-17437 added support for specifying a WAL directory outside the HBase root directory or even in a different FileSystem since 1.3.3/2.0+. Some FileSystems (such as Amazon S3) don’t support append or consistent writes, in such scenario WAL directory needs to be configured in a different FileSystem to avoid loss of writes.</p>
</div>
<div class="paragraph">
<p>Following configurations are added to accomplish this:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><code>hbase.wal.dir</code></p>
<div class="paragraph">
<p>This defines where the root WAL directory is located, could be on a different FileSystem than the root directory. WAL directory can not be set to a subdirectory of the root directory. The default value of this is the root directory if unset.</p>
</div>
</li>
<li>
<p><code>hbase.rootdir.perms</code></p>
<div class="paragraph">
<p>Configures FileSystem permissions to set on the root directory. This is '700' by default.</p>
</div>
</li>
<li>
<p><code>hbase.wal.dir.perms</code></p>
<div class="paragraph">
<p>Configures FileSystem permissions to set on the WAL directory FileSystem. This is '700' by default.</p>
</div>
</li>
</ol>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
While migrating to custom WAL dir (outside the HBase root directory or a different FileSystem) existing WAL files must be copied manually to new WAL dir, otherwise it may lead to data loss/inconsistency as HMaster has no information about previous WAL directory.
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="wal.disable"><a class="anchor" href="#wal.disable"></a>71.6.10. Disabling the WAL</h4>
<div class="paragraph">
<p>It is possible to disable the WAL, to improve performance in certain specific situations.
However, disabling the WAL puts your data at risk.
The only situation where this is recommended is during a bulk load.
This is because, in the event of a problem, the bulk load can be re-run with no risk of data loss.</p>
</div>
<div class="paragraph">
<p>The WAL is disabled by calling the HBase client field <code>Mutation.writeToWAL(false)</code>.
Use the <code>Mutation.setDurability(Durability.SKIP_WAL)</code> and Mutation.getDurability() methods to set and get the field&#8217;s value.
There is no way to disable the WAL for only a specific table.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
If you disable the WAL for anything other than bulk loads, your data is at risk.
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="regions.arch"><a class="anchor" href="#regions.arch"></a>72. Regions</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Regions are the basic element of availability and distribution for tables, and are comprised of a Store per Column Family.
The hierarchy of objects is as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Table (HBase table)
Region (Regions for the table)
Store (Store per ColumnFamily for each Region for the table)
MemStore (MemStore for each Store for each Region for the table)
StoreFile (StoreFiles for each Store for each Region for the table)
Block (Blocks within a StoreFile within a Store for each Region for the table)</pre>
</div>
</div>
<div class="paragraph">
<p>For a description of what HBase files look like when written to HDFS, see <a href="#trouble.namenode.hbase.objects">Browsing HDFS for HBase Objects</a>.</p>
</div>
<div class="sect2">
<h3 id="arch.regions.size"><a class="anchor" href="#arch.regions.size"></a>72.1. Considerations for Number of Regions</h3>
<div class="paragraph">
<p>In general, HBase is designed to run with a small (20-200) number of relatively large (5-20Gb) regions per server.
The considerations for this are as follows:</p>
</div>
<div class="sect3">
<h4 id="too_many_regions"><a class="anchor" href="#too_many_regions"></a>72.1.1. Why should I keep my Region count low?</h4>
<div class="paragraph">
<p>Typically you want to keep your region count low on HBase for numerous reasons.
Usually right around 100 regions per RegionServer has yielded the best results.
Here are some of the reasons below for keeping region count low:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>MSLAB (MemStore-local allocation buffer) requires 2MB per MemStore (that&#8217;s 2MB per family per region). 1000 regions that have 2 families each is 3.9GB of heap used, and it&#8217;s not even storing data yet.
NB: the 2MB value is configurable.</p>
</li>
<li>
<p>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.
Rewriting the same data tens of times is the last thing you want.
An example is filling 1000 regions (with one family) equally and let&#8217;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.
5MB inserted later, it would flush another region that will now have a bit over 5MB of data, and so on.
This is currently the main limiting factor for the number of regions; see <a href="#ops.capacity.regions.count">Number of regions per RS - upper bound</a> for detailed formula.</p>
</li>
<li>
<p>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.
The reason is that it&#8217;s heavy on ZK usage, and it&#8217;s not very async at the moment (could really be improved&#8201;&#8212;&#8201;and has been improved a bunch in 0.96 HBase).</p>
</li>
<li>
<p>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</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Another issue is the effect of the number of regions on MapReduce jobs; it is typical to have one mapper per HBase region.
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.</p>
</div>
<div class="paragraph">
<p>See <a href="#ops.capacity.regions">Determining region count and size</a> for configuration guidelines.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="regions.arch.assignment"><a class="anchor" href="#regions.arch.assignment"></a>72.2. Region-RegionServer Assignment</h3>
<div class="paragraph">
<p>This section describes how Regions are assigned to RegionServers.</p>
</div>
<div class="sect3">
<h4 id="regions.arch.assignment.startup"><a class="anchor" href="#regions.arch.assignment.startup"></a>72.2.1. Startup</h4>
<div class="paragraph">
<p>When HBase starts regions are assigned as follows (short version):</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The Master invokes the <code>AssignmentManager</code> upon startup.</p>
</li>
<li>
<p>The <code>AssignmentManager</code> looks at the existing region assignments in <code>hbase:meta</code>.</p>
</li>
<li>
<p>If the region assignment is still valid (i.e., if the RegionServer is still online) then the assignment is kept.</p>
</li>
<li>
<p>If the assignment is invalid, then the <code>LoadBalancerFactory</code> is invoked to assign the region.
The load balancer (<code>StochasticLoadBalancer</code> by default in HBase 1.0) assign the region to a RegionServer.</p>
</li>
<li>
<p><code>hbase:meta</code> 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.</p>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="regions.arch.assignment.failover"><a class="anchor" href="#regions.arch.assignment.failover"></a>72.2.2. Failover</h4>
<div class="paragraph">
<p>When a RegionServer fails:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The regions immediately become unavailable because the RegionServer is down.</p>
</li>
<li>
<p>The Master will detect that the RegionServer has failed.</p>
</li>
<li>
<p>The region assignments will be considered invalid and will be re-assigned just like the startup sequence.</p>
</li>
<li>
<p>In-flight queries are re-tried, and not lost.</p>
</li>
<li>
<p>Operations are switched to a new RegionServer within the following amount of time:</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">ZooKeeper session timeout + split time + assignment/replay time</code></pre>
</div>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="regions.arch.balancer"><a class="anchor" href="#regions.arch.balancer"></a>72.2.3. Region Load Balancing</h4>
<div class="paragraph">
<p>Regions can be periodically moved by the <a href="#master.processes.loadbalancer">LoadBalancer</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="regions.arch.states"><a class="anchor" href="#regions.arch.states"></a>72.2.4. Region State Transition</h4>
<div class="paragraph">
<p>HBase maintains a state for each region and persists the state in <code>hbase:meta</code>.
The state of the <code>hbase:meta</code> region itself is persisted in ZooKeeper.
You can see the states of regions in transition in the Master web UI.
Following is the list of possible region states.</p>
</div>
<div class="ulist">
<div class="title">Possible Region States</div>
<ul>
<li>
<p><code>OFFLINE</code>: the region is offline and not opening</p>
</li>
<li>
<p><code>OPENING</code>: the region is in the process of being opened</p>
</li>
<li>
<p><code>OPEN</code>: the region is open and the RegionServer has notified the master</p>
</li>
<li>
<p><code>FAILED_OPEN</code>: the RegionServer failed to open the region</p>
</li>
<li>
<p><code>CLOSING</code>: the region is in the process of being closed</p>
</li>
<li>
<p><code>CLOSED</code>: the RegionServer has closed the region and notified the master</p>
</li>
<li>
<p><code>FAILED_CLOSE</code>: the RegionServer failed to close the region</p>
</li>
<li>
<p><code>SPLITTING</code>: the RegionServer notified the master that the region is splitting</p>
</li>
<li>
<p><code>SPLIT</code>: the RegionServer notified the master that the region has finished splitting</p>
</li>
<li>
<p><code>SPLITTING_NEW</code>: this region is being created by a split which is in progress</p>
</li>
<li>
<p><code>MERGING</code>: the RegionServer notified the master that this region is being merged with another region</p>
</li>
<li>
<p><code>MERGED</code>: the RegionServer notified the master that this region has been merged</p>
</li>
<li>
<p><code>MERGING_NEW</code>: this region is being created by a merge of two regions</p>
</li>
</ul>
</div>
<div class="imageblock">
<div class="content">
<img src="images/region_states.png" alt="region states">
</div>
<div class="title">Figure 3. Region State Transitions</div>
</div>
<div class="ulist">
<div class="title">Graph Legend</div>
<ul>
<li>
<p>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)</p>
</li>
<li>
<p>Palegreen: Online state that regions can serve requests</p>
</li>
<li>
<p>Lightblue: Transient states</p>
</li>
<li>
<p>Red: Failure states that need OPS attention</p>
</li>
<li>
<p>Gold: Terminal states of regions split/merged</p>
</li>
<li>
<p>Grey: Initial states of regions created through split/merge</p>
</li>
</ul>
</div>
<div class="olist arabic">
<div class="title">Transition State Descriptions</div>
<ol class="arabic">
<li>
<p>The master moves a region from <code>OFFLINE</code> to <code>OPENING</code> state and tries to assign the region to a RegionServer.
The RegionServer may or may not have received the open region request.
The master retries sending the open region request to the RegionServer until the RPC goes through or the master runs out of retries.
After the RegionServer receives the open region request, the RegionServer begins opening the region.</p>
</li>
<li>
<p>If the master is running out of retries, the master prevents the RegionServer from opening the region by moving the region to <code>CLOSING</code> state and trying to close it, even if the RegionServer is starting to open the region.</p>
</li>
<li>
<p>After the RegionServer opens the region, it continues to try to notify the master until the master moves the region to <code>OPEN</code> state and notifies the RegionServer.
The region is now open.</p>
</li>
<li>
<p>If the RegionServer cannot open the region, it notifies the master.
The master moves the region to <code>CLOSED</code> state and tries to open the region on a different RegionServer.</p>
</li>
<li>
<p>If the master cannot open the region on any of a certain number of regions, it moves the region to <code>FAILED_OPEN</code> state, and takes no further action until an operator intervenes from the HBase shell, or the server is dead.</p>
</li>
<li>
<p>The master moves a region from <code>OPEN</code> to <code>CLOSING</code> state.
The RegionServer holding the region may or may not have received the close region request.
The master retries sending the close request to the server until the RPC goes through or the master runs out of retries.</p>
</li>
<li>
<p>If the RegionServer is not online, or throws <code>NotServingRegionException</code>, the master moves the region to <code>OFFLINE</code> state and re-assigns it to a different RegionServer.</p>
</li>
<li>
<p>If the RegionServer is online, but not reachable after the master runs out of retries, the master moves the region to <code>FAILED_CLOSE</code> state and takes no further action until an operator intervenes from the HBase shell, or the server is dead.</p>
</li>
<li>
<p>If the RegionServer gets the close region request, it closes the region and notifies the master.
The master moves the region to <code>CLOSED</code> state and re-assigns it to a different RegionServer.</p>
</li>
<li>
<p>Before assigning a region, the master moves the region to <code>OFFLINE</code> state automatically if it is in <code>CLOSED</code> state.</p>
</li>
<li>
<p>When a RegionServer is about to split a region, it notifies the master.
The master moves the region to be split from <code>OPEN</code> to <code>SPLITTING</code> state and add the two new regions to be created to the RegionServer.
These two regions are in <code>SPLITTING_NEW</code> state initially.</p>
</li>
<li>
<p>After notifying the master, the RegionServer starts to split the region.
Once past the point of no return, the RegionServer notifies the master again so the master can update the <code>hbase:meta</code> table.
However, the master does not update the region states until it is notified by the server that the split is done.
If the split is successful, the splitting region is moved from <code>SPLITTING</code> to <code>SPLIT</code> state and the two new regions are moved from <code>SPLITTING_NEW</code> to <code>OPEN</code> state.</p>
</li>
<li>
<p>If the split fails, the splitting region is moved from <code>SPLITTING</code> back to <code>OPEN</code> state, and the two new regions which were created are moved from <code>SPLITTING_NEW</code> to <code>OFFLINE</code> state.</p>
</li>
<li>
<p>When a RegionServer is about to merge two regions, it notifies the master first.
The master moves the two regions to be merged from <code>OPEN</code> to <code>MERGING</code> state, and adds the new region which will hold the contents of the merged regions region to the RegionServer.
The new region is in <code>MERGING_NEW</code> state initially.</p>
</li>
<li>
<p>After notifying the master, the RegionServer starts to merge the two regions.
Once past the point of no return, the RegionServer notifies the master again so the master can update the META.
However, the master does not update the region states until it is notified by the RegionServer that the merge has completed.
If the merge is successful, the two merging regions are moved from <code>MERGING</code> to <code>MERGED</code> state and the new region is moved from <code>MERGING_NEW</code> to <code>OPEN</code> state.</p>
</li>
<li>
<p>If the merge fails, the two merging regions are moved from <code>MERGING</code> back to <code>OPEN</code> state, and the new region which was created to hold the contents of the merged regions is moved from <code>MERGING_NEW</code> to <code>OFFLINE</code> state.</p>
</li>
<li>
<p>For regions in <code>FAILED_OPEN</code> or <code>FAILED_CLOSE</code> states, the master tries to close them again when they are reassigned by an operator via HBase Shell.</p>
</li>
</ol>
</div>
</div>
</div>
<div class="sect2">
<h3 id="regions.arch.locality"><a class="anchor" href="#regions.arch.locality"></a>72.3. Region-RegionServer Locality</h3>
<div class="paragraph">
<p>Over time, Region-RegionServer locality is achieved via HDFS block replication.
The HDFS client does the following by default when choosing locations to write replicas:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>First replica is written to local node</p>
</li>
<li>
<p>Second replica is written to a random node on another rack</p>
</li>
<li>
<p>Third replica is written on the same rack as the second, but on a different node chosen randomly</p>
</li>
<li>
<p>Subsequent replicas are written on random nodes on the cluster.
See <em>Replica Placement: The First Baby Steps</em> on this page: <a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">HDFS Architecture</a></p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Thus, HBase eventually achieves locality for a region after a flush or a compaction.
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.</p>
</div>
<div class="paragraph">
<p>For more information, see <em>Replica Placement: The First Baby Steps</em> on this page: <a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">HDFS Architecture</a> and also Lars George&#8217;s blog on <a href="http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html">HBase and HDFS locality</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.region.splits"><a class="anchor" href="#arch.region.splits"></a>72.4. Region Splits</h3>
<div class="paragraph">
<p>Regions split when they reach a configured threshold.
Below we treat the topic in short.
For a longer exposition, see <a href="http://hortonworks.com/blog/apache-hbase-region-splitting-and-merging/">Apache HBase Region Splitting and Merging</a> by our Enis Soztutar.</p>
</div>
<div class="paragraph">
<p>Splits run unaided on the RegionServer; i.e. the Master does not participate.
The RegionServer splits a region, offlines the split region and then adds the daughter regions to <code>hbase:meta</code>, opens daughters on the parent&#8217;s hosting RegionServer and then reports the split to the Master.
See <a href="#disable.splitting">Managed Splitting</a> for how to manually manage splits (and for why you might do this).</p>
</div>
<div class="sect3">
<h4 id="_custom_split_policies"><a class="anchor" href="#_custom_split_policies"></a>72.4.1. Custom Split Policies</h4>
<div class="paragraph">
<p>You can override the default split policy using a custom
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html">RegionSplitPolicy</a>(HBase 0.94+).
Typically a custom split policy should extend HBase&#8217;s default split policy:
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html">IncreasingToUpperBoundRegionSplitPolicy</a>.</p>
</div>
<div class="paragraph">
<p>The policy can set globally through the HBase configuration or on a per-table
basis.</p>
</div>
<div class="listingblock">
<div class="title">Configuring the Split Policy Globally in <em>hbase-site.xml</em></div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.region.split.policy<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="title">Configuring a Split Policy On a Table Using the Java API</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HTableDescriptor tableDesc = <span class="keyword">new</span> HTableDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">test</span><span class="delimiter">&quot;</span></span>);
tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
tableDesc.addFamily(<span class="keyword">new</span> HColumnDescriptor(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">cf1</span><span class="delimiter">&quot;</span></span>)));
admin.createTable(tableDesc);
----</code></pre>
</div>
</div>
<div class="listingblock">
<div class="title">Configuring the Split Policy On a Table Using HBase Shell</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; create <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, {METADATA =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">SPLIT_POLICY</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy</span><span class="delimiter">'</span></span>}},{NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">cf1</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
<div class="paragraph">
<p>The policy can be set globally through the HBaseConfiguration used or on a per table basis:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HTableDescriptor myHtd = ...;
myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName());</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The <code>DisabledRegionSplitPolicy</code> policy blocks manual region splitting.
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="manual_region_splitting_decisions"><a class="anchor" href="#manual_region_splitting_decisions"></a>72.5. Manual Region Splitting</h3>
<div class="paragraph">
<p>It is possible to manually split your table, either at table creation (pre-splitting), or at a later time as an administrative action.
You might choose to split your region for one or more of the following reasons.
There may be other valid reasons, but the need to manually split your table might also point to problems with your schema design.</p>
</div>
<div class="ulist">
<div class="title">Reasons to Manually Split Your Table</div>
<ul>
<li>
<p>Your data is sorted by timeseries or another similar algorithm that sorts new data at the end of the table.
This means that the Region Server holding the last region is always under load, and the other Region Servers are idle, or mostly idle.
See also <a href="#timeseries">Monotonically Increasing Row Keys/Timeseries Data</a>.</p>
</li>
<li>
<p>You have developed an unexpected hotspot in one region of your table.
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.
See <a href="#perf.one.region">perf.one.region</a> for more discussion about this particular scenario.</p>
</li>
<li>
<p>After a big increase in the number of RegionServers in your cluster, to get the load spread out quickly.</p>
</li>
<li>
<p>Before a bulk-load which is likely to cause unusual and uneven load across regions.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>See <a href="#disable.splitting">Managed Splitting</a> for a discussion about the dangers and possible benefits of managing splitting completely manually.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The <code>DisabledRegionSplitPolicy</code> policy blocks manual region splitting.
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="_determining_split_points"><a class="anchor" href="#_determining_split_points"></a>72.5.1. Determining Split Points</h4>
<div class="paragraph">
<p>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&#8217;t get you there.
Keeping that in mind, the way you split your regions is very dependent upon the characteristics of your data.
It may be that you already know the best way to split your table.
If not, the way you split your table depends on what your keys are like.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Alphanumeric Rowkeys</dt>
<dd>
<p>If your rowkeys start with a letter or number, you can split your table at letter or number boundaries.
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.</p>
</dd>
<dt class="hdlist1">Using a Custom Algorithm</dt>
<dd>
<p>The RegionSplitter tool is provided with HBase, and uses a <em>SplitAlgorithm</em> to determine split points for you.
As parameters, you give it the algorithm, desired number of regions, and column families.
It includes three split algorithms.
The first is the
<code><a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.HexStringSplit.html">HexStringSplit</a></code>
algorithm, which assumes the row keys are hexadecimal strings.
The second is the
<code><a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.DecimalStringSplit.html">DecimalStringSplit</a></code>
algorithm, which assumes the row keys are decimal strings in the range 00000000 to 99999999.
The third,
<code><a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.UniformSplit.html">UniformSplit</a></code>,
assumes the row keys are random byte arrays.
You will probably need to develop your own
<code><a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.SplitAlgorithm.html">SplitAlgorithm</a></code>,
using the provided ones as models.</p>
</dd>
</dl>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_online_region_merges"><a class="anchor" href="#_online_region_merges"></a>72.6. Online Region Merges</h3>
<div class="paragraph">
<p>Both Master and RegionServer participate in the event of online region merges.
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.
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 <code>hbase:meta</code> and adds the merged region to <code>hbase:meta</code>, opens the merged region on the RegionServer and reports the merge to the Master.</p>
</div>
<div class="paragraph">
<p>An example of region merges in the HBase shell</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ hbase&gt; merge_region 'ENCODED_REGIONNAME', 'ENCODED_REGIONNAME'
$ hbase&gt; merge_region 'ENCODED_REGIONNAME', 'ENCODED_REGIONNAME', true</code></pre>
</div>
</div>
<div class="paragraph">
<p>It&#8217;s an asynchronous operation and call returns immediately without waiting merge completed.
Passing <code>true</code> as the optional third parameter will force a merge. Normally only adjacent regions can be merged.
The <code>force</code> parameter overrides this behaviour and is for expert use only.</p>
</div>
</div>
<div class="sect2">
<h3 id="store"><a class="anchor" href="#store"></a>72.7. Store</h3>
<div class="paragraph">
<p>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.</p>
</div>
<div class="sect3">
<h4 id="store.memstore"><a class="anchor" href="#store.memstore"></a>72.7.1. MemStore</h4>
<div class="paragraph">
<p>The MemStore holds in-memory modifications to the Store.
Modifications are Cells/KeyValues.
When a flush is requested, the current MemStore is moved to a snapshot and is cleared.
HBase continues to serve edits from the new MemStore and backing snapshot until the flusher reports that the flush succeeded.
At this point, the snapshot is discarded.
Note that when the flush happens, MemStores that belong to the same region will all be flushed.</p>
</div>
</div>
<div class="sect3">
<h4 id="_memstore_flush"><a class="anchor" href="#_memstore_flush"></a>72.7.2. MemStore Flush</h4>
<div class="paragraph">
<p>A MemStore flush can be triggered under any of the conditions listed below.
The minimum flush unit is per region, not at individual MemStore level.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>When a MemStore reaches the size specified by <code>hbase.hregion.memstore.flush.size</code>,
all MemStores that belong to its region will be flushed out to disk.</p>
</li>
<li>
<p>When the overall MemStore usage reaches the value specified by
<code>hbase.regionserver.global.memstore.upperLimit</code>, MemStores from various regions
will be flushed out to disk to reduce overall MemStore usage in a RegionServer.</p>
<div class="paragraph">
<p>The flush order is based on the descending order of a region&#8217;s MemStore usage.</p>
</div>
<div class="paragraph">
<p>Regions will have their MemStores flushed until the overall MemStore usage drops
to or slightly below <code>hbase.regionserver.global.memstore.lowerLimit</code>.</p>
</div>
</li>
<li>
<p>When the number of WAL log entries in a given region server&#8217;s WAL reaches the
value specified in <code>hbase.regionserver.max.logs</code>, MemStores from various regions
will be flushed out to disk to reduce the number of logs in the WAL.</p>
<div class="paragraph">
<p>The flush order is based on time.</p>
</div>
<div class="paragraph">
<p>Regions with the oldest MemStores are flushed first until WAL count drops below
<code>hbase.regionserver.max.logs</code>.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="hregion.scans"><a class="anchor" href="#hregion.scans"></a>72.7.3. Scans</h4>
<div class="ulist">
<ul>
<li>
<p>When a client issues a scan against a table, HBase generates <code>RegionScanner</code> objects, one per region, to serve the scan request.</p>
</li>
<li>
<p>The <code>RegionScanner</code> object contains a list of <code>StoreScanner</code> objects, one per column family.</p>
</li>
<li>
<p>Each <code>StoreScanner</code> object further contains a list of <code>StoreFileScanner</code> objects, corresponding to each StoreFile and HFile of the corresponding column family, and a list of <code>KeyValueScanner</code> objects for the MemStore.</p>
</li>
<li>
<p>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.</p>
</li>
<li>
<p>When a <code>StoreFileScanner</code> object is constructed, it is associated with a <code>MultiVersionConcurrencyControl</code> read point, which is the current <code>memstoreTS</code>, filtering out any new updates beyond the read point.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="hfile"><a class="anchor" href="#hfile"></a>72.7.4. StoreFile (HFile)</h4>
<div class="paragraph">
<p>StoreFiles are where your data lives.</p>
</div>
<div class="sect4">
<h5 id="_hfile_format"><a class="anchor" href="#_hfile_format"></a>HFile Format</h5>
<div class="paragraph">
<p>The <em>HFile</em> file format is based on the SSTable file described in the <a href="http://research.google.com/archive/bigtable.html">BigTable [2006</a>] paper and on Hadoop&#8217;s <a href="https://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/file/tfile/TFile.html">TFile</a> (The unit test suite and the compression harness were taken directly from TFile). Schubert Zhang&#8217;s blog post on <a href="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</a> makes for a thorough introduction to HBase&#8217;s HFile.
Matteo Bertozzi has also put up a helpful description, <a href="http://th30z.blogspot.com/2011/02/hbase-io-hfile.html?spref=tw">HBase I/O: HFile</a>.</p>
</div>
<div class="paragraph">
<p>For more information, see the HFile source code.
Also see <a href="#hfilev2">HBase file format with inline blocks (version 2)</a> for information about the HFile v2 format that was included in 0.92.</p>
</div>
</div>
<div class="sect4">
<h5 id="hfile_tool"><a class="anchor" href="#hfile_tool"></a>HFile Tool</h5>
<div class="paragraph">
<p>To view a textualized version of HFile content, you can use the <code>hbase hfile</code> tool.
Type the following to see usage:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ${HBASE_HOME}/bin/hbase hfile</code></pre>
</div>
</div>
<div class="paragraph">
<p>For example, to view the content of the file <em>hdfs://10.81.47.41:8020/hbase/default/TEST/1418428042/DSMP/4759508618286845475</em>, type the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash"> $ ${HBASE_HOME}/bin/hbase hfile -v -f hdfs://10.81.47.41:8020/hbase/default/TEST/1418428042/DSMP/4759508618286845475</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you leave off the option -v to see just a summary on the HFile.
See usage for other things to do with the <code>hfile</code> tool.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
In the output of this tool, you might see 'seqid=0' for certain keys in places such as 'Mid-key'/'firstKey'/'lastKey'. These are
'KeyOnlyKeyValue' type instances - meaning their seqid is irrelevant &amp; we just need the keys of these Key-Value instances.
</td>
</tr>
</table>
</div>
</div>
<div class="sect4">
<h5 id="store.file.dir"><a class="anchor" href="#store.file.dir"></a>StoreFile Directory Structure on HDFS</h5>
<div class="paragraph">
<p>For more information of what StoreFiles look like on HDFS with respect to the directory structure, see <a href="#trouble.namenode.hbase.objects">Browsing HDFS for HBase Objects</a>.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="hfile.blocks"><a class="anchor" href="#hfile.blocks"></a>72.7.5. Blocks</h4>
<div class="paragraph">
<p>StoreFiles are composed of blocks.
The blocksize is configured on a per-ColumnFamily basis.</p>
</div>
<div class="paragraph">
<p>Compression happens at the block level within StoreFiles.
For more information on compression, see <a href="#compression">Compression and Data Block Encoding In HBase</a>.</p>
</div>
<div class="paragraph">
<p>For more information on blocks, see the HFileBlock source code.</p>
</div>
</div>
<div class="sect3">
<h4 id="keyvalue"><a class="anchor" href="#keyvalue"></a>72.7.6. KeyValue</h4>
<div class="paragraph">
<p>The KeyValue class is the heart of data storage in HBase.
KeyValue wraps a byte array and takes offsets and lengths into the passed array which specify where to start interpreting the content as KeyValue.</p>
</div>
<div class="paragraph">
<p>The KeyValue format inside a byte array is:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>keylength</p>
</li>
<li>
<p>valuelength</p>
</li>
<li>
<p>key</p>
</li>
<li>
<p>value</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The Key is further decomposed as:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>rowlength</p>
</li>
<li>
<p>row (i.e., the rowkey)</p>
</li>
<li>
<p>columnfamilylength</p>
</li>
<li>
<p>columnfamily</p>
</li>
<li>
<p>columnqualifier</p>
</li>
<li>
<p>timestamp</p>
</li>
<li>
<p>keytype (e.g., Put, Delete, DeleteColumn, DeleteFamily)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>KeyValue instances are <em>not</em> split across blocks.
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.
For more information, see the KeyValue source code.</p>
</div>
<div class="sect4">
<h5 id="keyvalue.example"><a class="anchor" href="#keyvalue.example"></a>Example</h5>
<div class="paragraph">
<p>To emphasize the points above, examine what happens with two Puts for two different columns for the same row:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Put #1: <code>rowkey=row1, cf:attr1=value1</code></p>
</li>
<li>
<p>Put #2: <code>rowkey=row1, cf:attr2=value2</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Even though these are for the same row, a KeyValue is created for each column:</p>
</div>
<div class="paragraph">
<p>Key portion for Put #1:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>rowlength -----------&#8594; 4</code></p>
</li>
<li>
<p><code>row -----------------&#8594; row1</code></p>
</li>
<li>
<p><code>columnfamilylength --&#8594; 2</code></p>
</li>
<li>
<p><code>columnfamily --------&#8594; cf</code></p>
</li>
<li>
<p><code>columnqualifier -----&#8594; attr1</code></p>
</li>
<li>
<p><code>timestamp -----------&#8594; server time of Put</code></p>
</li>
<li>
<p><code>keytype -------------&#8594; Put</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Key portion for Put #2:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>rowlength -----------&#8594; 4</code></p>
</li>
<li>
<p><code>row -----------------&#8594; row1</code></p>
</li>
<li>
<p><code>columnfamilylength --&#8594; 2</code></p>
</li>
<li>
<p><code>columnfamily --------&#8594; cf</code></p>
</li>
<li>
<p><code>columnqualifier -----&#8594; attr2</code></p>
</li>
<li>
<p><code>timestamp -----------&#8594; server time of Put</code></p>
</li>
<li>
<p><code>keytype -------------&#8594; Put</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>It is critical to understand that the rowkey, ColumnFamily, and column (aka columnqualifier) are embedded within the KeyValue instance.
The longer these identifiers are, the bigger the KeyValue is.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="compaction"><a class="anchor" href="#compaction"></a>72.7.7. Compaction</h4>
<div class="ulist">
<div class="title">Ambiguous Terminology</div>
<ul>
<li>
<p>A <em>StoreFile</em> is a facade of HFile.
In terms of compaction, use of StoreFile seems to have prevailed in the past.</p>
</li>
<li>
<p>A <em>Store</em> is the same thing as a ColumnFamily.
StoreFiles are related to a Store, or ColumnFamily.</p>
</li>
<li>
<p>If you want to read more about StoreFiles versus HFiles and Stores versus ColumnFamilies, see <a href="https://issues.apache.org/jira/browse/HBASE-11316">HBASE-11316</a>.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>When the MemStore reaches a given size (<code>hbase.hregion.memstore.flush.size</code>), it flushes its contents to a StoreFile.
The number of StoreFiles in a Store increases over time. <em>Compaction</em> is an operation which reduces the number of StoreFiles in a Store, by merging them together, in order to increase performance on read operations.
Compactions can be resource-intensive to perform, and can either help or hinder performance depending on many factors.</p>
</div>
<div class="paragraph">
<p>Compactions fall into two categories: minor and major.
Minor and major compactions differ in the following ways.</p>
</div>
<div class="paragraph">
<p><em>Minor compactions</em> usually select a small number of small, adjacent StoreFiles and rewrite them as a single StoreFile.
Minor compactions do not drop (filter out) deletes or expired versions, because of potential side effects.
See <a href="#compaction.and.deletes">Compaction and Deletions</a> and <a href="#compaction.and.versions">Compaction and Versions</a> for information on how deletes and versions are handled in relation to compactions.
The end result of a minor compaction is fewer, larger StoreFiles for a given Store.</p>
</div>
<div class="paragraph">
<p>The end result of a <em>major compaction</em> is a single StoreFile per Store.
Major compactions also process delete markers and max versions.
See <a href="#compaction.and.deletes">Compaction and Deletions</a> and <a href="#compaction.and.versions">Compaction and Versions</a> for information on how deletes and versions are handled in relation to compactions.</p>
</div>
<div id="compaction.and.deletes" class="paragraph">
<div class="title">Compaction and Deletions</div>
<p>When an explicit deletion occurs in HBase, the data is not actually deleted.
Instead, a <em>tombstone</em> marker is written.
The tombstone marker prevents the data from being returned with queries.
During a major compaction, the data is actually deleted, and the tombstone marker is removed from the StoreFile.
If the deletion happens because of an expired TTL, no tombstone is created.
Instead, the expired data is filtered out and is not written back to the compacted StoreFile.</p>
</div>
<div id="compaction.and.versions" class="paragraph">
<div class="title">Compaction and Versions</div>
<p>When you create a Column Family, you can specify the maximum number of versions to keep, by specifying <code>ColumnFamilyDescriptorBuilder.setMaxVersions(int versions)</code>.
The default value is <code>1</code>.
If more versions than the specified maximum exist, the excess versions are filtered out and not written back to the compacted StoreFile.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Major Compactions Can Impact Query Results</div>
<div class="paragraph">
<p>In some situations, older versions can be inadvertently resurrected if a newer version is explicitly deleted.
See <a href="#major.compactions.change.query.results">Major compactions change query results</a> for a more in-depth explanation.
This situation is only possible before the compaction finishes.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>In theory, major compactions improve performance.
However, on a highly loaded system, major compactions can require an inappropriate number of resources and adversely affect performance.
In a default configuration, major compactions are scheduled automatically to run once in a 7-day period.
This is sometimes inappropriate for systems in production.
You can manage major compactions manually.
See <a href="#managed.compactions">Managed Compactions</a>.</p>
</div>
<div class="paragraph">
<p>Compactions do not perform region merges.
See <a href="#ops.regionmgt.merge">Merge</a> for more information on region merging.</p>
</div>
<div class="paragraph">
<div class="title">Compaction Switch</div>
<p>We can switch on and off the compactions at region servers. Switching off compactions will also
interrupt any currently ongoing compactions. It can be done dynamically using the "compaction_switch"
command from hbase shell. If done from the command line, this setting will be lost on restart of the
server. To persist the changes across region servers modify the configuration hbase.regionserver
.compaction.enabled in hbase-site.xml and restart HBase.</p>
</div>
<div class="sect4">
<h5 id="compaction.file.selection"><a class="anchor" href="#compaction.file.selection"></a>Compaction Policy - HBase 0.96.x and newer</h5>
<div class="paragraph">
<p>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.
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 <em>compaction policy</em>.</p>
</div>
<div class="paragraph">
<p>Prior to HBase 0.96.x, there was only one compaction policy.
That original compaction policy is still available as <code>RatioBasedCompactionPolicy</code>. The new compaction default policy, called <code>ExploringCompactionPolicy</code>, was subsequently backported to HBase 0.94 and HBase 0.95, and is the default in HBase 0.96 and newer.
It was implemented in <a href="https://issues.apache.org/jira/browse/HBASE-7842">HBASE-7842</a>.
In short, <code>ExploringCompactionPolicy</code> attempts to select the best possible set of StoreFiles to compact with the least amount of work, while the <code>RatioBasedCompactionPolicy</code> selects the first set that meets the criteria.</p>
</div>
<div class="paragraph">
<p>Regardless of the compaction policy used, file selection is controlled by several configurable parameters and happens in a multi-step approach.
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.</p>
</div>
<div class="sect5">
<h6 id="compaction.being.stuck"><a class="anchor" href="#compaction.being.stuck"></a>Being Stuck</h6>
<div class="paragraph">
<p>When the MemStore gets too large, it needs to flush its contents to a StoreFile.
However, Stores are configured with a bound on the number StoreFiles,
<code>hbase.hstore.blockingStoreFiles</code>, and if in excess, the MemStore flush must wait
until the StoreFile count is reduced by one or more compactions. If the MemStore
is too large and the number of StoreFiles is also too high, the algorithm is said
to be "stuck". By default we&#8217;ll wait on compactions up to
<code>hbase.hstore.blockingWaitTime</code> milliseconds. If this period expires, we&#8217;ll flush
anyways even though we are in excess of the
<code>hbase.hstore.blockingStoreFiles</code> count.</p>
</div>
<div class="paragraph">
<p>Upping the <code>hbase.hstore.blockingStoreFiles</code> count will allow flushes to happen
but a Store with many StoreFiles in will likely have higher read latencies. Try to
figure why Compactions are not keeping up. Is it a write spurt that is bringing
about this situation or is a regular occurance and the cluster is under-provisioned
for the volume of writes?</p>
</div>
</div>
<div class="sect5">
<h6 id="exploringcompaction.policy"><a class="anchor" href="#exploringcompaction.policy"></a>The ExploringCompactionPolicy Algorithm</h6>
<div class="paragraph">
<p>The ExploringCompactionPolicy algorithm considers each possible set of adjacent StoreFiles before choosing the set where compaction will have the most benefit.</p>
</div>
<div class="paragraph">
<p>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.
This can "trick" HBase into choosing to perform a major compaction each time a compaction is needed, and cause a lot of extra overhead.
With the ExploringCompactionPolicy, major compactions happen much less frequently because minor compactions are more efficient.</p>
</div>
<div class="paragraph">
<p>In general, ExploringCompactionPolicy is the right choice for most situations, and thus is the default compaction policy.
You can also use ExploringCompactionPolicy along with <a href="#ops.stripe">Experimental: Stripe Compactions</a>.</p>
</div>
<div class="paragraph">
<p>The logic of this policy can be examined in hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java.
The following is a walk-through of the logic of the ExploringCompactionPolicy.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Make a list of all existing StoreFiles in the Store.
The rest of the algorithm filters this list to come up with the subset of HFiles which will be chosen for compaction.</p>
</li>
<li>
<p>If this was a user-requested compaction, attempt to perform the requested compaction type, regardless of what would normally be chosen.
Note that even if the user requests a major compaction, it may not be possible to perform a major compaction.
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.</p>
</li>
<li>
<p>Some StoreFiles are automatically excluded from consideration.
These include:</p>
<div class="ulist">
<ul>
<li>
<p>StoreFiles that are larger than <code>hbase.hstore.compaction.max.size</code></p>
</li>
<li>
<p>StoreFiles that were created by a bulk-load operation which explicitly excluded compaction.
You may decide to exclude StoreFiles resulting from bulk loads, from compaction.
To do this, specify the <code>hbase.mapreduce.hfileoutputformat.compaction.exclude</code> parameter during the bulk load operation.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Iterate through the list from step 1, and make a list of all potential sets of StoreFiles to compact together.
A potential set is a grouping of <code>hbase.hstore.compaction.min</code> contiguous StoreFiles in the list.
For each set, perform some sanity-checking and figure out whether this is the best compaction that could be done:</p>
<div class="ulist">
<ul>
<li>
<p>If the number of StoreFiles in this set (not the size of the StoreFiles) is fewer than <code>hbase.hstore.compaction.min</code> or more than <code>hbase.hstore.compaction.max</code>, take it out of consideration.</p>
</li>
<li>
<p>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.
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.
See <a href="#compaction.being.stuck">Being Stuck</a>.</p>
</li>
<li>
<p>Do size-based sanity checks against each StoreFile in this set of StoreFiles.</p>
<div class="ulist">
<ul>
<li>
<p>If the size of this StoreFile is larger than <code>hbase.hstore.compaction.max.size</code>, take it out of consideration.</p>
</li>
<li>
<p>If the size is greater than or equal to <code>hbase.hstore.compaction.min.size</code>, sanity-check it against the file-based ratio to see whether it is too large to be considered.</p>
<div class="paragraph">
<p>The sanity-checking is successful if:</p>
</div>
</li>
<li>
<p>There is only one StoreFile in this set, or</p>
</li>
<li>
<p>For each StoreFile, its size multiplied by <code>hbase.hstore.compaction.ratio</code> (or <code>hbase.hstore.compaction.ratio.offpeak</code> 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.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</li>
<li>
<p>If this set of StoreFiles is still in consideration, compare it to the previously-selected best compaction.
If it is better, replace the previously-selected best compaction with this one.</p>
</li>
<li>
<p>When the entire list of potential compactions has been processed, perform the best compaction that was found.
If no StoreFiles were selected for compaction, but there are multiple StoreFiles, assume the algorithm is stuck (see <a href="#compaction.being.stuck">Being Stuck</a>) and if so, perform the smallest compaction that was found in step 3.</p>
</li>
</ol>
</div>
</div>
<div class="sect5">
<h6 id="compaction.ratiobasedcompactionpolicy.algorithm"><a class="anchor" href="#compaction.ratiobasedcompactionpolicy.algorithm"></a>RatioBasedCompactionPolicy Algorithm</h6>
<div class="paragraph">
<p>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.
To use the RatioBasedCompactionPolicy rather than the ExploringCompactionPolicy, set <code>hbase.hstore.defaultengine.compactionpolicy.class</code> to <code>RatioBasedCompactionPolicy</code> in the <em>hbase-site.xml</em> file.
To switch back to the ExploringCompactionPolicy, remove the setting from the <em>hbase-site.xml</em>.</p>
</div>
<div class="paragraph">
<p>The following section walks you through the algorithm used to select StoreFiles for compaction in the RatioBasedCompactionPolicy.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The first phase is to create a list of all candidates for compaction.
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.
This list of StoreFiles is ordered by the sequence ID.
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.</p>
</li>
<li>
<p>Check to see if the algorithm is stuck (see <a href="#compaction.being.stuck">Being Stuck</a>, and if so, a major compaction is forced.
This is a key area where <a href="#exploringcompaction.policy">The ExploringCompactionPolicy Algorithm</a> is often a better choice than the RatioBasedCompactionPolicy.</p>
</li>
<li>
<p>If the compaction was user-requested, try to perform the type of compaction that was requested.
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 <code>hbase.hstore.compaction.max</code>).</p>
</li>
<li>
<p>Some StoreFiles are automatically excluded from consideration.
These include:</p>
<div class="ulist">
<ul>
<li>
<p>StoreFiles that are larger than <code>hbase.hstore.compaction.max.size</code></p>
</li>
<li>
<p>StoreFiles that were created by a bulk-load operation which explicitly excluded compaction.
You may decide to exclude StoreFiles resulting from bulk loads, from compaction.
To do this, specify the <code>hbase.mapreduce.hfileoutputformat.compaction.exclude</code> parameter during the bulk load operation.</p>
</li>
</ul>
</div>
</li>
<li>
<p>The maximum number of StoreFiles allowed in a major compaction is controlled by the <code>hbase.hstore.compaction.max</code> parameter.
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.
However, a user-requested major compaction still occurs even if there are more than <code>hbase.hstore.compaction.max</code> StoreFiles to compact.</p>
</li>
<li>
<p>If the list contains fewer than <code>hbase.hstore.compaction.min</code> StoreFiles to compact, a minor compaction is aborted.
Note that a major compaction can be performed on a single HFile.
Its function is to remove deletes and expired versions, and reset locality on the StoreFile.</p>
</li>
<li>
<p>The value of the <code>hbase.hstore.compaction.ratio</code> 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.
For instance, if hbase.hstore.compaction.ratio is 1.2, FileX is 5MB, FileY is 2MB, and FileZ is 3MB:</p>
<div class="listingblock">
<div class="content">
<pre>5 &lt;= 1.2 x (2 + 3) or 5 &lt;= 6</pre>
</div>
</div>
<div class="paragraph">
<p>In this scenario, FileX is eligible for minor compaction.
If FileX were 7MB, it would not be eligible for minor compaction.
This ratio favors smaller StoreFile.
You can configure a different ratio for use in off-peak hours, using the parameter <code>hbase.hstore.compaction.ratio.offpeak</code>, if you also configure <code>hbase.offpeak.start.hour</code> and <code>hbase.offpeak.end.hour</code>.</p>
</div>
</li>
<li>
<p>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.
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.
Prior to HBase 0.96, the major compaction period was 24 hours.
See <code>hbase.hregion.majorcompaction</code> in the table below to tune or disable time-based major compactions.</p>
</li>
</ol>
</div>
</div>
<div class="sect5">
<h6 id="compaction.parameters"><a class="anchor" href="#compaction.parameters"></a>Parameters Used by Compaction Algorithm</h6>
<div class="paragraph">
<p>This table contains the main configuration parameters for compaction.
This list is not exhaustive.
To tune these parameters from the defaults, edit the <em>hbase-default.xml</em> file.
For a full list of all configuration parameters available, see <a href="#config.files">config.files</a></p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>hbase.hstore.compaction.min</code></dt>
<dd>
<p>The minimum number of StoreFiles which must be eligible for compaction before compaction can run.
The goal of tuning <code>hbase.hstore.compaction.min</code> is to avoid ending up with too many tiny StoreFiles
to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles
in a Store, and this is probably not appropriate. If you set this value too high, all the other
values will need to be adjusted accordingly. For most cases, the default value is appropriate.
In previous versions of HBase, the parameter <code>hbase.hstore.compaction.min</code> was called
<code>hbase.hstore.compactionThreshold</code>.</p>
<div class="paragraph">
<p><strong>Default</strong>: 3</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hstore.compaction.max</code></dt>
<dd>
<p>The maximum number of StoreFiles which will be selected for a single minor compaction,
regardless of the number of eligible StoreFiles. Effectively, the value of
<code>hbase.hstore.compaction.max</code> controls the length of time it takes a single
compaction to complete. Setting it larger means that more StoreFiles are included
in a compaction. For most cases, the default value is appropriate.</p>
<div class="paragraph">
<p><strong>Default</strong>: 10</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hstore.compaction.min.size</code></dt>
<dd>
<p>A StoreFile smaller than this size will always be eligible for minor compaction.
StoreFiles this size or larger are evaluated by <code>hbase.hstore.compaction.ratio</code>
to determine if they are eligible. Because this limit represents the "automatic
include" limit for all StoreFiles smaller than this value, this value may need
to be reduced in write-heavy environments where many files in the 1-2 MB range
are being flushed, because every StoreFile will be targeted for compaction and
the resulting StoreFiles may still be under the minimum size and require further
compaction. If this parameter is lowered, the ratio check is triggered more quickly.
This addressed some issues seen in earlier versions of HBase but changing this
parameter is no longer necessary in most situations.</p>
<div class="paragraph">
<p><strong>Default</strong>:128 MB</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hstore.compaction.max.size</code></dt>
<dd>
<p>A StoreFile larger than this size will be excluded from compaction. The effect of
raising <code>hbase.hstore.compaction.max.size</code> is fewer, larger StoreFiles that do not
get compacted often. If you feel that compaction is happening too often without
much benefit, you can try raising this value.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>Long.MAX_VALUE</code></p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hstore.compaction.ratio</code></dt>
<dd>
<p>For minor compaction, this ratio is used to determine whether a given StoreFile
which is larger than <code>hbase.hstore.compaction.min.size</code> is eligible for compaction.
Its effect is to limit compaction of large StoreFile. The value of
<code>hbase.hstore.compaction.ratio</code> is expressed as a floating-point decimal.</p>
<div class="ulist">
<ul>
<li>
<p>A large ratio, such as 10, will produce a single giant StoreFile. Conversely,
a value of .25, will produce behavior similar to the BigTable compaction algorithm,
producing four StoreFiles.</p>
</li>
<li>
<p>A moderate value of between 1.0 and 1.4 is recommended. When tuning this value,
you are balancing write costs with read costs. Raising the value (to something like
1.4) will have more write costs, because you will compact larger StoreFiles.
However, during reads, HBase will need to seek through fewer StoreFiles to
accomplish the read. Consider this approach if you cannot take advantage of <a href="#blooms">Bloom Filters</a>.</p>
</li>
<li>
<p>Alternatively, you can lower this value to something like 1.0 to reduce the
background cost of writes, and use to limit the number of StoreFiles touched
during reads. For most cases, the default value is appropriate.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>1.2F</code></p>
</div>
</li>
</ul>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hstore.compaction.ratio.offpeak</code></dt>
<dd>
<p>The compaction ratio used during off-peak compactions, if off-peak hours are
also configured (see below). Expressed as a floating-point decimal. This allows
for more aggressive (or less aggressive, if you set it lower than
<code>hbase.hstore.compaction.ratio</code>) compaction during a set time period. Ignored
if off-peak is disabled (default). This works the same as
<code>hbase.hstore.compaction.ratio</code>.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>5.0F</code></p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.offpeak.start.hour</code></dt>
<dd>
<p>The start of off-peak hours, expressed as an integer between 0 and 23, inclusive.
Set to -1 to disable off-peak.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>-1</code> (disabled)</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.offpeak.end.hour</code></dt>
<dd>
<p>The end of off-peak hours, expressed as an integer between 0 and 23, inclusive.
Set to -1 to disable off-peak.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>-1</code> (disabled)</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.regionserver.thread.compaction.throttle</code></dt>
<dd>
<p>There are two different thread pools for compactions, one for large compactions
and the other for small compactions. This helps to keep compaction of lean tables
(such as <code>hbase:meta</code>) fast. If a compaction is larger than this threshold,
it goes into the large compaction pool. In most cases, the default value is
appropriate.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size</code>
(which defaults to <code>128</code>)</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hregion.majorcompaction</code></dt>
<dd>
<p>Time between major compactions, expressed in milliseconds. Set to 0 to disable
time-based automatic major compactions. User-requested and size-based major
compactions will still run. This value is multiplied by
<code>hbase.hregion.majorcompaction.jitter</code> to cause compaction to start at a
somewhat-random time during a given window of time.</p>
<div class="paragraph">
<p><strong>Default</strong>: 7 days (<code>604800000</code> milliseconds)</p>
</div>
</dd>
<dt class="hdlist1"><code>hbase.hregion.majorcompaction.jitter</code></dt>
<dd>
<p>A multiplier applied to hbase.hregion.majorcompaction to cause compaction to
occur a given amount of time either side of <code>hbase.hregion.majorcompaction</code>.
The smaller the number, the closer the compactions will happen to the
<code>hbase.hregion.majorcompaction</code> interval. Expressed as a floating-point decimal.</p>
<div class="paragraph">
<p><strong>Default</strong>: <code>.50F</code></p>
</div>
</dd>
</dl>
</div>
</div>
</div>
<div class="sect4">
<h5 id="compaction.file.selection.old"><a class="anchor" href="#compaction.file.selection.old"></a>Compaction File Selection</h5>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Legacy Information</div>
<div class="paragraph">
<p>This section has been preserved for historical reasons and refers to the way compaction worked prior to HBase 0.96.x.
You can still use this behavior if you enable <a href="#compaction.ratiobasedcompactionpolicy.algorithm">RatioBasedCompactionPolicy Algorithm</a>. For information on the way that compactions work in HBase 0.96.x and later, see <a href="#compaction">Compaction</a>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>To understand the core algorithm for StoreFile selection, there is some ASCII-art in the Store source code that will serve as useful reference.</p>
</div>
<div class="paragraph">
<p>It has been copied below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">/* normal skew:
*
* older ----&gt; newer
* _
* | | _
* | | | | _
* --|-|- |-|- |-|---_-------_------- minCompactSize
* | | | | | | | | _ | |
* | | | | | | | | | | | |
* | | | | | | | | | | | |
*/</span></code></pre>
</div>
</div>
<div class="ulist">
<div class="title">Important knobs:</div>
<ul>
<li>
<p><code>hbase.hstore.compaction.ratio</code> Ratio used in compaction file selection algorithm (default 1.2f).</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min</code> (in HBase v 0.90 this is called <code>hbase.hstore.compactionThreshold</code>) (files) Minimum number of StoreFiles per Store to be selected for a compaction to occur (default 2).</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max</code> (files) Maximum number of StoreFiles to compact per minor compaction (default 10).</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min.size</code> (bytes) Any StoreFile smaller than this setting with automatically be a candidate for compaction.
Defaults to <code>hbase.hregion.memstore.flush.size</code> (128 mb).</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max.size</code> (.92) (bytes) Any StoreFile larger than this setting with automatically be excluded from compaction (default Long.MAX_VALUE).</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The minor compaction StoreFile selection logic is size based, and selects a file for compaction when the <code>file &#8656; sum(smaller_files) * hbase.hstore.compaction.ratio</code>.</p>
</div>
<div class="sect5">
<h6 id="compaction.file.selection.example1"><a class="anchor" href="#compaction.file.selection.example1"></a>Minor Compaction File Selection - Example #1 (Basic Example)</h6>
<div class="paragraph">
<p>This example mirrors an example from the unit test <code>TestCompactSelection</code>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.hstore.compaction.ratio</code> = 1.0f</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min</code> = 3 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max</code> = 5 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min.size</code> = 10 (bytes)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>Why?</p>
</div>
<div class="ulist">
<ul>
<li>
<p>100 &#8594; No, because sum(50, 23, 12, 12) * 1.0 = 97.</p>
</li>
<li>
<p>50 &#8594; No, because sum(23, 12, 12) * 1.0 = 47.</p>
</li>
<li>
<p>23 &#8594; Yes, because sum(12, 12) * 1.0 = 24.</p>
</li>
<li>
<p>12 &#8594; Yes, because the previous file has been included, and because this does not exceed the max-file limit of 5</p>
</li>
<li>
<p>12 &#8594; Yes, because the previous file had been included, and because this does not exceed the max-file limit of 5.</p>
</li>
</ul>
</div>
</div>
<div class="sect5">
<h6 id="compaction.file.selection.example2"><a class="anchor" href="#compaction.file.selection.example2"></a>Minor Compaction File Selection - Example #2 (Not Enough Files ToCompact)</h6>
<div class="paragraph">
<p>This example mirrors an example from the unit test <code>TestCompactSelection</code>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.hstore.compaction.ratio</code> = 1.0f</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min</code> = 3 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max</code> = 5 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min.size</code> = 10 (bytes)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following StoreFiles exist: 100, 25, 12, and 12 bytes apiece (oldest to newest). With the above parameters, no compaction will be started.</p>
</div>
<div class="paragraph">
<p>Why?</p>
</div>
<div class="ulist">
<ul>
<li>
<p>100 &#8594; No, because sum(25, 12, 12) * 1.0 = 47</p>
</li>
<li>
<p>25 &#8594; No, because sum(12, 12) * 1.0 = 24</p>
</li>
<li>
<p>12 &#8594; No. Candidate because sum(12) * 1.0 = 12, there are only 2 files to compact and that is less than the threshold of 3</p>
</li>
<li>
<p>12 &#8594; No. Candidate because the previous StoreFile was, but there are not enough files to compact</p>
</li>
</ul>
</div>
</div>
<div class="sect5">
<h6 id="compaction.file.selection.example3"><a class="anchor" href="#compaction.file.selection.example3"></a>Minor Compaction File Selection - Example #3 (Limiting Files To Compact)</h6>
<div class="paragraph">
<p>This example mirrors an example from the unit test <code>TestCompactSelection</code>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.hstore.compaction.ratio</code> = 1.0f</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min</code> = 3 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max</code> = 5 (files)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.min.size</code> = 10 (bytes)</p>
</li>
<li>
<p><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>Why?</p>
</div>
<div class="ulist">
<ul>
<li>
<p>7 &#8594; Yes, because sum(6, 5, 4, 3, 2, 1) * 1.0 = 21.
Also, 7 is less than the min-size</p>
</li>
<li>
<p>6 &#8594; Yes, because sum(5, 4, 3, 2, 1) * 1.0 = 15.
Also, 6 is less than the min-size.</p>
</li>
<li>
<p>5 &#8594; Yes, because sum(4, 3, 2, 1) * 1.0 = 10.
Also, 5 is less than the min-size.</p>
</li>
<li>
<p>4 &#8594; Yes, because sum(3, 2, 1) * 1.0 = 6.
Also, 4 is less than the min-size.</p>
</li>
<li>
<p>3 &#8594; Yes, because sum(2, 1) * 1.0 = 3.
Also, 3 is less than the min-size.</p>
</li>
<li>
<p>2 &#8594; No.
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.</p>
</li>
<li>
<p>1 &#8594; No.
Candidate because previous file was selected and 1 is less than the min-size, but max-number of files to compact has been reached.</p>
</li>
</ul>
</div>
<div id="compaction.config.impact" class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Impact of Key Configuration Options</div>
This information is now included in the configuration parameter table in <a href="#compaction.parameters">Parameters Used by Compaction Algorithm</a>.
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect4">
<h5 id="ops.date.tiered"><a class="anchor" href="#ops.date.tiered"></a>Date Tiered Compaction</h5>
<div class="paragraph">
<p>Date tiered compaction is a date-aware store file compaction strategy that is beneficial for time-range scans for time-series data.</p>
</div>
</div>
<div class="sect4">
<h5 id="ops.date.tiered.when"><a class="anchor" href="#ops.date.tiered.when"></a>When To Use Date Tiered Compactions</h5>
<div class="paragraph">
<p>Consider using Date Tiered Compaction for reads for limited time ranges, especially scans of recent data</p>
</div>
<div class="paragraph">
<p>Don&#8217;t use it for</p>
</div>
<div class="ulist">
<ul>
<li>
<p>random gets without a limited time range</p>
</li>
<li>
<p>frequent deletes and updates</p>
</li>
<li>
<p>Frequent out of order data writes creating long tails, especially writes with future timestamps</p>
</li>
<li>
<p>frequent bulk loads with heavily overlapping time ranges</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Performance Improvements</div>
<p>Performance testing has shown that the performance of time-range scans improve greatly for limited time ranges, especially scans of recent data.</p>
</div>
<div class="sect5">
<h6 id="ops.date.tiered.enable"><a class="anchor" href="#ops.date.tiered.enable"></a>Enabling Date Tiered Compaction</h6>
<div class="paragraph">
<p>You can enable Date Tiered compaction for a table or a column family, by setting its <code>hbase.hstore.engine.class</code> to <code>org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine</code>.</p>
</div>
<div class="paragraph">
<p>You also need to set <code>hbase.hstore.blockingStoreFiles</code> 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</p>
</div>
<div class="paragraph">
<p>You also need to set <code>hbase.hstore.compaction.max</code> to the same value as <code>hbase.hstore.blockingStoreFiles</code> to unblock major compaction.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Enable Date Tiered Compaction</div>
<ol class="arabic">
<li>
<p>Run one of following commands in the HBase shell.
Replace the table name <code>orders_table</code> with the name of your table.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.min</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">2</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.max</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>}
<span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, {NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">blobs_cf</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.min</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">2</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.max</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>}}
<span class="class">create</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">blobs_cf</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.min</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">2</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.max</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">60</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
</li>
<li>
<p>Configure other options if needed.
See <a href="#ops.date.tiered.config">Configuring Date Tiered Compaction</a> for more information.</p>
</li>
</ol>
</div>
<div class="olist arabic">
<div class="title">Procedure: Disable Date Tiered Compaction</div>
<ol class="arabic">
<li>
<p>Set the <code>hbase.hstore.engine.class</code> option to either nil or <code>org.apache.hadoop.hbase.regionserver.DefaultStoreEngine</code>.
Either option has the same effect.
Make sure you set the other options you changed to the original settings too.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.DefaultStoreEngine</span><span class="delimiter">'</span></span><span class="error"></span> <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">12</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.min</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">6</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.compaction.max</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">12</span><span class="delimiter">'</span></span>}}</code></pre>
</div>
</div>
</li>
</ol>
</div>
<div class="paragraph">
<p>When you change the store engine either way, a major compaction will likely be performed on most regions.
This is not necessary on new tables.</p>
</div>
</div>
<div class="sect5">
<h6 id="ops.date.tiered.config"><a class="anchor" href="#ops.date.tiered.config"></a>Configuring Date Tiered Compaction</h6>
<div class="paragraph">
<p>Each of the settings for date tiered compaction should be configured at the table or column family level.
If you use HBase shell, the general command pattern is as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">key</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, ..., <span class="string"><span class="delimiter">'</span><span class="content">key</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>}}</code></pre>
</div>
</div>
<div id="ops.date.tiered.config.parameters" class="paragraph">
<div class="title">Tier Parameters</div>
<p>You can configure your date tiers by changing the settings for the following parameters:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 11. Date Tier Parameters</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Setting</th>
<th class="tableblock halign-left valign-top">Notes</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.hstore.compaction.date.tiered.max.storefile.age.millis</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Files with max-timestamp smaller than this will no longer be compacted.Default at Long.MAX_VALUE.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.hstore.compaction.date.tiered.base.window.millis</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Base window size in milliseconds. Default at 6 hours.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.hstore.compaction.date.tiered.windows.per.tier</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Number of windows per tier. Default at 4.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.hstore.compaction.date.tiered.incoming.window.min</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>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.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.hstore.compaction.date.tiered.window.policy.class</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>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.</p>
</div></div></td>
</tr>
</tbody>
</table>
<div id="ops.date.tiered.config.compaction.throttler" class="paragraph">
<div class="title">Compaction Throttler</div>
<p>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:
Set <code>hbase.regionserver.throughput.controller</code> to <code>org.apache.hadoop.hbase.regionserver.compactions.PressureAwareCompactionThroughputController</code>.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
For more information about date tiered compaction, please refer to the design specification at <a href="https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8" class="bare">https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8</a>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect4">
<h5 id="ops.stripe"><a class="anchor" href="#ops.stripe"></a>Experimental: Stripe Compactions</h5>
<div class="paragraph">
<p>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.
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.
The stripes are transparent to the rest of HBase, so other operations on the HFiles or data work without modification.</p>
</div>
<div class="paragraph">
<p>Stripe compactions change the HFile layout, creating sub-regions within regions.
These sub-regions are easier to compact, and should result in fewer major compactions.
This approach alleviates some of the challenges of larger regions.</p>
</div>
<div class="paragraph">
<p>Stripe compaction is fully compatible with <a href="#compaction">Compaction</a> and works in conjunction with either the ExploringCompactionPolicy or RatioBasedCompactionPolicy.
It can be enabled for existing tables, and the table will continue to operate normally if it is disabled later.</p>
</div>
</div>
<div class="sect4">
<h5 id="ops.stripe.when"><a class="anchor" href="#ops.stripe.when"></a>When To Use Stripe Compactions</h5>
<div class="paragraph">
<p>Consider using stripe compaction if you have either of the following:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Large regions.
You can get the positive effects of smaller regions without additional overhead for MemStore and region management overhead.</p>
</li>
<li>
<p>Non-uniform keys, such as time dimension in a key.
Only the stripes receiving the new keys will need to compact.
Old data will not compact as often, if at all</p>
</li>
</ul>
</div>
<div class="paragraph">
<div class="title">Performance Improvements</div>
<p>Performance testing has shown that the performance of reads improves somewhat, and variability of performance of reads and writes is greatly reduced.
An overall long-term performance improvement is seen on large non-uniform-row key regions, such as a hash-prefixed timestamp key.
These performance gains are the most dramatic on a table which is already large.
It is possible that the performance improvement might extend to region splits.</p>
</div>
<div class="sect5">
<h6 id="ops.stripe.enable"><a class="anchor" href="#ops.stripe.enable"></a>Enabling Stripe Compaction</h6>
<div class="paragraph">
<p>You can enable stripe compaction for a table or a column family, by setting its <code>hbase.hstore.engine.class</code> to <code>org.apache.hadoop.hbase.regionserver.StripeStoreEngine</code>.
You also need to set the <code>hbase.hstore.blockingStoreFiles</code> to a high number, such as 100 (rather than the default value of 10).</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Enable Stripe Compaction</div>
<ol class="arabic">
<li>
<p>Run one of following commands in the HBase shell.
Replace the table name <code>orders_table</code> with the name of your table.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.StripeStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">100</span><span class="delimiter">'</span></span>}
<span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, {NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">blobs_cf</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.StripeStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">100</span><span class="delimiter">'</span></span>}}
<span class="class">create</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">blobs_cf</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">org.apache.hadoop.hbase.regionserver.StripeStoreEngine</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.blockingStoreFiles</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">100</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
</li>
<li>
<p>Configure other options if needed.
See <a href="#ops.stripe.config">Configuring Stripe Compaction</a> for more information.</p>
</li>
<li>
<p>Enable the table.</p>
</li>
</ol>
</div>
<div class="olist arabic">
<div class="title">Procedure: Disable Stripe Compaction</div>
<ol class="arabic">
<li>
<p>Set the <code>hbase.hstore.engine.class</code> option to either nil or <code>org.apache.hadoop.hbase.regionserver.DefaultStoreEngine</code>.
Either option has the same effect.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">hbase.hstore.engine.class</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">rg.apache.hadoop.hbase.regionserver.DefaultStoreEngine</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
</li>
<li>
<p>Enable the table.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>When you enable a large table after changing the store engine either way, a major compaction will likely be performed on most regions.
This is not necessary on new tables.</p>
</div>
</div>
<div class="sect5">
<h6 id="ops.stripe.config"><a class="anchor" href="#ops.stripe.config"></a>Configuring Stripe Compaction</h6>
<div class="paragraph">
<p>Each of the settings for stripe compaction should be configured at the table or column family level.
If you use HBase shell, the general command pattern is as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="sql"><span class="class">alter</span> <span class="string"><span class="delimiter">'</span><span class="content">orders_table</span><span class="delimiter">'</span></span>, CONFIGURATION =&gt; {<span class="string"><span class="delimiter">'</span><span class="content">key</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>, ..., <span class="string"><span class="delimiter">'</span><span class="content">key</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">value</span><span class="delimiter">'</span></span>}}</code></pre>
</div>
</div>
<div id="ops.stripe.config.sizing" class="paragraph">
<div class="title">Region and stripe sizing</div>
<p>You can configure your stripe sizing based upon your region sizing.
By default, your new regions will start with one stripe.
On the next compaction after the stripe has grown too large (16 x MemStore flushes size), it is split into two stripes.
Stripe splitting continues as the region grows, until the region is large enough to split.</p>
</div>
<div class="paragraph">
<p>You can improve this pattern for your own data.
A good rule is to aim for a stripe size of at least 1 GB, and about 8-12 stripes for uniform row keys.
For example, if your regions are 30 GB, 12 x 2.5 GB stripes might be a good starting point.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 12. Stripe Sizing Settings</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Setting</th>
<th class="tableblock halign-left valign-top">Notes</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.store.stripe.initialStripeCount</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>The number of stripes to create when stripe compaction is enabled. You can use it as follows:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>For relatively uniform row keys, if you know the approximate
target number of stripes from the above, you can avoid some
splitting overhead by starting with several stripes (2, 5, 10&#8230;&#8203;).
If the early data is not representative of overall row key
distribution, this will not be as efficient.</p>
</li>
<li>
<p>For existing tables with a large amount of data, this setting
will effectively pre-split your stripes.</p>
</li>
<li>
<p>For keys such as hash-prefixed sequential keys, with more than
one hash prefix per region, pre-splitting may make sense.</p>
</li>
</ul>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.store.stripe.sizeToSplit</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>The maximum size a stripe grows before splitting. Use this in
conjunction with <code>hbase.store.stripe.splitPartCount</code> to
control the target stripe size (<code>sizeToSplit = splitPartsCount * target
stripe size</code>), according to the above sizing considerations.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>hbase.store.stripe.splitPartCount</code></p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>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.</p>
</div></div></td>
</tr>
</tbody>
</table>
<div id="ops.stripe.config.memstore" class="paragraph">
<div class="title">MemStore Size Settings</div>
<p>By default, the flush creates several files from one MemStore, according to existing stripe boundaries and row keys to flush.
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.</p>
</div>
<div class="paragraph">
<p>In this type of situation, you can set <code>hbase.store.stripe.compaction.flushToL0</code> to <code>true</code>.
This will cause a MemStore flush to create a single file instead.
When at least <code>hbase.store.stripe.compaction.minFilesL0</code> such files (by default, 4) accumulate, they will be compacted into striped files.</p>
</div>
<div id="ops.stripe.config.compact" class="paragraph">
<div class="title">Normal Compaction Configuration and Stripe Compaction</div>
<p>All the settings that apply to normal compactions (see <a href="#compaction.parameters">Parameters Used by Compaction Algorithm</a>) apply to stripe compactions.
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.
To control these for stripe compactions, use <code>hbase.store.stripe.compaction.minFiles</code> and <code>hbase.store.stripe.compaction.maxFiles</code>, rather than <code>hbase.hstore.compaction.min</code> and <code>hbase.hstore.compaction.max</code>.</p>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="arch.bulk.load"><a class="anchor" href="#arch.bulk.load"></a>73. Bulk Loading</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="arch.bulk.load.overview"><a class="anchor" href="#arch.bulk.load.overview"></a>73.1. Overview</h3>
<div class="paragraph">
<p>HBase includes several methods of loading data into tables.
The most straightforward method is to either use the <code>TableOutputFormat</code> class from a MapReduce job, or use the normal client APIs; however, these are not always the most efficient methods.</p>
</div>
<div class="paragraph">
<p>The bulk load feature uses a MapReduce job to output table data in HBase&#8217;s internal data format, and then directly loads the generated StoreFiles into a running cluster.
Using bulk load will use less CPU and network resources than simply using the HBase API.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.bulk.load.arch"><a class="anchor" href="#arch.bulk.load.arch"></a>73.2. Bulk Load Architecture</h3>
<div class="paragraph">
<p>The HBase bulk load process consists of two main steps.</p>
</div>
<div class="sect3">
<h4 id="arch.bulk.load.prep"><a class="anchor" href="#arch.bulk.load.prep"></a>73.2.1. Preparing data via a MapReduce job</h4>
<div class="paragraph">
<p>The first step of a bulk load is to generate HBase data files (StoreFiles) from a MapReduce job using <code>HFileOutputFormat2</code>.
This output format writes out data in HBase&#8217;s internal storage format so that they can be later loaded very efficiently into the cluster.</p>
</div>
<div class="paragraph">
<p>In order to function efficiently, <code>HFileOutputFormat2</code> must be configured such that each output HFile fits within a single region.
In order to do this, jobs whose output will be bulk loaded into HBase use Hadoop&#8217;s <code>TotalOrderPartitioner</code> class to partition the map output into disjoint ranges of the key space, corresponding to the key ranges of the regions in the table.</p>
</div>
<div class="paragraph">
<p><code>HFileOutputFormat2</code> includes a convenience function, <code>configureIncrementalLoad()</code>, which automatically sets up a <code>TotalOrderPartitioner</code> based on the current region boundaries of a table.</p>
</div>
</div>
<div class="sect3">
<h4 id="arch.bulk.load.complete"><a class="anchor" href="#arch.bulk.load.complete"></a>73.2.2. Completing the data load</h4>
<div class="paragraph">
<p>After a data import has been prepared, either by using the <code>importtsv</code> tool with the &#8220;importtsv.bulk.output&#8221; option or by some other MapReduce job using the <code>HFileOutputFormat</code>, the <code>completebulkload</code> tool is used to import the data into the running cluster.
This command line tool iterates through the prepared data files, and for each one determines the region the file belongs to.
It then contacts the appropriate RegionServer which adopts the HFile, moving it into its storage directory and making the data available to clients.</p>
</div>
<div class="paragraph">
<p>If the region boundaries have changed during the course of bulk load preparation, or between the preparation and completion steps, the <code>completebulkload</code> utility will automatically split the data files into pieces corresponding to the new boundaries.
This process is not optimally efficient, so users should take care to minimize the delay between preparing a bulk load and importing it into the cluster, especially if other clients are simultaneously loading data through other means.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ hadoop jar hbase-server-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable</code></pre>
</div>
</div>
<div class="paragraph">
<p>The <code>-c config-file</code> 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).</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If the target table does not already exist in HBase, this tool will create the table automatically.
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="arch.bulk.load.also"><a class="anchor" href="#arch.bulk.load.also"></a>73.3. See Also</h3>
<div class="paragraph">
<p>For more information about the referenced utilities, see <a href="#importtsv">ImportTsv</a> and <a href="#completebulkload">CompleteBulkLoad</a>.</p>
</div>
<div class="paragraph">
<p>See <a href="http://blog.cloudera.com/blog/2013/09/how-to-use-hbase-bulk-loading-and-why/">How-to: Use HBase Bulk Loading, and Why</a> for a recent blog on current state of bulk loading.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.bulk.load.adv"><a class="anchor" href="#arch.bulk.load.adv"></a>73.4. Advanced Usage</h3>
<div class="paragraph">
<p>Although the <code>importtsv</code> tool is useful in many cases, advanced users may want to generate data programmatically, or import data from other formats.
To get started doing so, dig into <code>ImportTsv.java</code> and check the JavaDoc for HFileOutputFormat.</p>
</div>
<div class="paragraph">
<p>The import step of the bulk load can also be done programmatically.
See the <code>LoadIncrementalHFiles</code> class for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.bulk.load.replication"><a class="anchor" href="#arch.bulk.load.replication"></a>73.5. Bulk Loading Replication</h3>
<div class="paragraph">
<p>HBASE-13153 adds replication support for bulk loaded HFiles, available since HBase 1.3/2.0. This feature is enabled by setting <code>hbase.replication.bulkload.enabled</code> to <code>true</code> (default is <code>false</code>).
You also need to copy the source cluster configuration files to the destination cluster.</p>
</div>
<div class="paragraph">
<p>Additional configurations are required too:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><code>hbase.replication.source.fs.conf.provider</code></p>
<div class="paragraph">
<p>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 <code>org.apache.hadoop.hbase.replication.regionserver.DefaultSourceFSConfigurationProvider</code>.</p>
</div>
</li>
<li>
<p><code>hbase.replication.conf.dir</code></p>
<div class="paragraph">
<p>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 <code>$HBASE_CONF_DIR</code>.</p>
</div>
</li>
<li>
<p><code>hbase.replication.cluster.id</code></p>
<div class="paragraph">
<p>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.</p>
</div>
</li>
</ol>
</div>
<div class="paragraph">
<p>For example: If source cluster FS client configurations are copied to the destination cluster under directory <code>/home/user/dc1/</code>, then <code>hbase.replication.cluster.id</code> should be configured as <code>dc1</code> and <code>hbase.replication.conf.dir</code> as <code>/home/user</code>.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<code>DefaultSourceFSConfigurationProvider</code> supports only <code>xml</code> 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.
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="arch.hdfs"><a class="anchor" href="#arch.hdfs"></a>74. HDFS</h2>
<div class="sectionbody">
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>See the Hadoop documentation on <a href="https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">HDFS Architecture</a> for more information.</p>
</div>
<div class="sect2">
<h3 id="arch.hdfs.nn"><a class="anchor" href="#arch.hdfs.nn"></a>74.1. NameNode</h3>
<div class="paragraph">
<p>The NameNode is responsible for maintaining the filesystem metadata.
See the above HDFS Architecture link for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="arch.hdfs.dn"><a class="anchor" href="#arch.hdfs.dn"></a>74.2. DataNode</h3>
<div class="paragraph">
<p>The DataNodes are responsible for storing HDFS blocks.
See the above HDFS Architecture link for more information.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="arch.timelineconsistent.reads"><a class="anchor" href="#arch.timelineconsistent.reads"></a>75. Timeline-consistent High Available Reads</h2>
<div class="sectionbody">
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The current <a href="#amv2">Assignment Manager V2</a> does not work well with region replica, so this feature maybe broken. Use it with caution.
</td>
</tr>
</table>
</div>
<div class="sect2">
<h3 id="casestudies.timelineconsistent.intro"><a class="anchor" href="#casestudies.timelineconsistent.intro"></a>75.1. Introduction</h3>
<div class="paragraph">
<p>HBase, architecturally, always had the strong consistency guarantee from the start.
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.</p>
</div>
<div class="paragraph">
<p>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.
There are three phases in the region recovery process - detection, assignment, and recovery.
Of these, the detection is usually the longest and is presently in the order of 20-30 seconds depending on the ZooKeeper session timeout.
During this time and before the recovery is complete, the clients will not be able to read the region data.</p>
</div>
<div class="paragraph">
<p>However, for some use cases, either the data may be read-only, or doing reads against some stale data is acceptable.
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.</p>
</div>
<div class="paragraph">
<p>For achieving high availability for reads, HBase provides a feature called <em>region replication</em>. In this model, for each region of a table, there will be multiple replicas that are opened in different RegionServers.
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.
If region replication is set to 2 or more, then the master will assign replicas of the regions of the table.
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).</p>
</div>
<div class="paragraph">
<p>All of the replicas for a single region will have a unique replica_id, starting from 0.
The region replica having replica_id==0 is called the primary region, and the others <em>secondary regions</em> or secondaries.
Only the primary can accept writes from the client, and the primary will always contain the latest changes.
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).</p>
</div>
</div>
<div class="sect2">
<h3 id="_timeline_consistency"><a class="anchor" href="#_timeline_consistency"></a>75.2. Timeline Consistency</h3>
<div class="paragraph">
<p>With this feature, HBase introduces a Consistency definition, which can be provided per read operation (get or scan).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">enum</span> Consistency {
STRONG,
TIMELINE
}</code></pre>
</div>
</div>
<div class="paragraph">
<p><code>Consistency.STRONG</code> is the default consistency model provided by HBase.
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.</p>
</div>
<div class="paragraph">
<p>In case a read is performed with <code>Consistency.TIMELINE</code>, then the read RPC will be sent to the primary region server first.
After a short interval (<code>hbase.client.primaryCallTimeout.get</code>, 10ms by default), parallel RPC for secondary region replicas will also be sent if the primary does not respond back.
After this, the result is returned from whichever RPC is finished first.
If the response came back from the primary region replica, we can always know that the data is latest.
For this Result.isStale() API has been added to inspect the staleness.
If the result is from a secondary region, then Result.isStale() will be set to true.
The user can then inspect this field to possibly reason about the data.</p>
</div>
<div class="paragraph">
<p>In terms of semantics, TIMELINE consistency as implemented by HBase differs from pure eventual consistency in these respects:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>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.
This replica is responsible for ordering the edits and preventing conflicts.
This guarantees that two different writes are not committed at the same time by different replicas and the data diverges.
With this, there is no need to do read-repair or last-timestamp-wins kind of conflict resolution.</p>
</li>
<li>
<p>The secondaries also apply the edits in the order that the primary committed them.
This way the secondaries will contain a snapshot of the primaries data at any point in time.
This is similar to RDBMS replications and even HBase&#8217;s own multi-datacenter replication, however in a single cluster.</p>
</li>
<li>
<p>On the read side, the client can detect whether the read is coming from up-to-date data or is stale data.
Also, the client can issue reads with different consistency requirements on a per-operation basis to ensure its own semantic guarantees.</p>
</li>
<li>
<p>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.
There is no stickiness to region replicas or a transaction-id based guarantee.
If required, this can be implemented later though.</p>
</li>
</ul>
</div>
<div class="imageblock">
<div class="content">
<img src="images/timeline_consistency.png" alt="Timeline Consistency">
</div>
<div class="title">Figure 4. Timeline Consistency</div>
</div>
<div class="paragraph">
<p>To better understand the TIMELINE semantics, let&#8217;s look at the above diagram.
Let&#8217;s say that there are two clients, and the first one writes x=1 at first, then x=2 and x=3 later.
As above, all writes are handled by the primary region replica.
The writes are saved in the write ahead log (WAL), and replicated to the other replicas asynchronously.
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.</p>
</div>
<div class="paragraph">
<p>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.
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.
Thus the client can see either 1, 2 or 3 as the value of x.
Let&#8217;s say that the primary region has failed and log replication cannot continue for some time.
If the client does multiple reads with TIMELINE consistency, she can observe x=2 first, then x=1, and so on.</p>
</div>
</div>
<div class="sect2">
<h3 id="_tradeoffs"><a class="anchor" href="#_tradeoffs"></a>75.3. Tradeoffs</h3>
<div class="paragraph">
<p>Having secondary regions hosted for read availability comes with some tradeoffs which should be carefully evaluated per use case.
Following are advantages and disadvantages.</p>
</div>
<div class="ulist">
<div class="title">Advantages</div>
<ul>
<li>
<p>High availability for read-only tables</p>
</li>
<li>
<p>High availability for stale reads</p>
</li>
<li>
<p>Ability to do very low latency reads with very high percentile (99.9%+) latencies for stale reads</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Disadvantages</div>
<ul>
<li>
<p>Double / Triple MemStore usage (depending on region replication count) for tables with region replication &gt; 1</p>
</li>
<li>
<p>Increased block cache usage</p>
</li>
<li>
<p>Extra network traffic for log replication</p>
</li>
<li>
<p>Extra backup RPCs for replicas</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>To serve the region data from multiple replicas, HBase opens the regions in secondary mode in the region servers.
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.</p>
</div>
</div>
<div class="sect2">
<h3 id="_where_is_the_code"><a class="anchor" href="#_where_is_the_code"></a>75.4. Where is the code</h3>
<div class="paragraph">
<p>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.</p>
</div>
</div>
<div class="sect2">
<h3 id="_propagating_writes_to_region_replicas"><a class="anchor" href="#_propagating_writes_to_region_replicas"></a>75.5. Propagating writes to region replicas</h3>
<div class="paragraph">
<p>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 <strong>only</strong> one of the following mechanisms: storefile refresher, or async wal replication. The latter is recommended.</p>
</div>
<div class="sect3">
<h4 id="_storefile_refresher"><a class="anchor" href="#_storefile_refresher"></a>75.5.1. StoreFile Refresher</h4>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>For turning this feature on, you should configure <code>hbase.regionserver.storefile.refresh.period</code> to a non-zero value. See Configuration section below.</p>
</div>
</div>
<div class="sect3">
<h4 id="_asnyc_wal_replication"><a class="anchor" href="#_asnyc_wal_replication"></a>75.5.2. Asnyc WAL replication</h4>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>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.</p>
</div>
<div class="paragraph">
<p>Async WAL Replication is <strong>disabled</strong> by default. You can enable this feature by setting <code>hbase.region.replica.replication.enabled</code> to <code>true</code>.
Asyn WAL Replication feature will add a new replication peer named <code>region_replica_replication</code> as a replication peer when you create a table with region replication &gt; 1 for the first time. Once enabled, if you want to disable this feature, you need to do two actions:
* Set configuration property <code>hbase.region.replica.replication.enabled</code> to false in <code>hbase-site.xml</code> (see Configuration section below)
* Disable the replication peer named <code>region_replica_replication</code> in the cluster using hbase shell or <code>Admin</code> class:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> hbase&gt; disable_peer 'region_replica_replication'</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_store_file_ttl"><a class="anchor" href="#_store_file_ttl"></a>75.6. Store File TTL</h3>
<div class="paragraph">
<p>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 <code>hbase.master.hfilecleaner.ttl</code> to a larger value, such as 1 hour to guarantee that you will not receive IOExceptions for requests going to replicas.</p>
</div>
</div>
<div class="sect2">
<h3 id="_region_replication_for_meta_table_s_region"><a class="anchor" href="#_region_replication_for_meta_table_s_region"></a>75.7. Region replication for META table’s region</h3>
<div class="paragraph">
<p>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 <code>hbase.regionserver.meta.storefile.refresh.period</code> needs to be set to a certain non-zero value for refreshing the meta store files. Note that this configuration is configured differently than
<code>hbase.regionserver.storefile.refresh.period</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_memory_accounting"><a class="anchor" href="#_memory_accounting"></a>75.8. Memory accounting</h3>
<div class="paragraph">
<p>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 <code>hbase.region.replica.storefile.refresh.memstore.multiplier</code> (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.</p>
</div>
</div>
<div class="sect2">
<h3 id="_secondary_replica_failover"><a class="anchor" href="#_secondary_replica_failover"></a>75.9. Secondary replica failover</h3>
<div class="paragraph">
<p>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&#8217;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 <code>hbase.region.replica.wait.for.primary.flush</code> (enabled by default) can be used to disable this feature if needed.</p>
</div>
</div>
<div class="sect2">
<h3 id="_configuration_properties"><a class="anchor" href="#_configuration_properties"></a>75.10. Configuration properties</h3>
<div class="paragraph">
<p>To use highly available reads, you should set the following properties in <code>hbase-site.xml</code> file.
There is no specific configuration to enable or disable region replicas.
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.</p>
</div>
<div class="sect3">
<h4 id="_server_side_properties"><a class="anchor" href="#_server_side_properties"></a>75.10.1. Server side properties</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.storefile.refresh.period<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>0<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.regionserver.meta.storefile.refresh.period<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>300000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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).
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.region.replica.replication.enabled<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named &quot;region_replica_replication&quot; will be created which will tail the logs and replicate the mutations to region replicas for tables that have region replication <span class="error">&gt;</span> 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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.region.replica.replication.memstore.enabled<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
If you set this to `false`, replicas do not receive memstore updates from
the primary RegionServer. If you set this to `true`, you can still disable
memstore replication on a per-table basis, by setting the table's
`REGION_MEMSTORE_REPLICATION` configuration property to `false`. If
memstore replication is disabled, the secondaries will only receive
updates for events like flushes and bulkloads, and will not have access to
data which the primary has not yet flushed. This preserves the guarantee
of row-level consistency, even when the read requests `Consistency.TIMELINE`.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.master.hfilecleaner.ttl<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3600000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
The period (in milliseconds) to keep store files in the archive folder before deleting them from the file system.<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.meta.replica.count<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Region replication count for the meta regions. Defaults to 1.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.region.replica.storefile.refresh.memstore.multiplier<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>4<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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).
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.region.replica.wait.for.primary.flush<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>One thing to keep in mind also is that, region replica placement policy is only enforced by the <code>StochasticLoadBalancer</code> which is the default balancer.
If you are using a custom load balancer property in hbase-site.xml (<code>hbase.master.loadbalancer.class</code>) replicas of regions might end up being hosted in the same server.</p>
</div>
</div>
<div class="sect3">
<h4 id="_client_side_properties"><a class="anchor" href="#_client_side_properties"></a>75.10.2. Client side properties</h4>
<div class="paragraph">
<p>Ensure to set the following for all clients (and servers) that will use region replicas.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.ipc.client.specificThreadForWriting<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Whether to enable interruption of RPC threads at the client side. This is required for region replicas with fallback RPC’s to secondary regions.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.primaryCallTimeout.get<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.primaryCallTimeout.multiget<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
The timeout (in microseconds), before secondary fallback RPC’s are submitted for multi-get requests (Table.get(List<span class="tag">&lt;Get&gt;</span>)) 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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.replicaCallTimeout.scan<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>1000000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
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.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.meta.replicas.use<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Whether to use meta table replicas or not. Default is false.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Note HBase-1.0.x users should use <code>hbase.ipc.client.allowsInterrupt</code> rather than <code>hbase.ipc.client.specificThreadForWriting</code>.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_user_interface"><a class="anchor" href="#_user_interface"></a>75.11. User Interface</h3>
<div class="paragraph">
<p>In the masters user interface, the region replicas of a table are also shown together with the primary regions.
You can notice that the replicas of a region will share the same start and end keys and the same region name prefix.
The only difference would be the appended replica_id (which is encoded as hex), and the region encoded name will be different.
You can also see the replica ids shown explicitly in the UI.</p>
</div>
</div>
<div class="sect2">
<h3 id="_creating_a_table_with_region_replication"><a class="anchor" href="#_creating_a_table_with_region_replication"></a>75.12. Creating a table with region replication</h3>
<div class="paragraph">
<p>Region replication is a per-table property.
All tables have <code>REGION_REPLICATION = 1</code> by default, which means that there is only one replica per region.
You can set and change the number of replicas per region of a table by supplying the <code>REGION_REPLICATION</code> property in the table descriptor.</p>
</div>
<div class="sect3">
<h4 id="_shell"><a class="anchor" href="#_shell"></a>75.12.1. Shell</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">create <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>, {REGION_REPLICATION =&gt; <span class="integer">2</span>}
describe <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>
<span class="keyword">for</span> i in <span class="integer">1</span>.<span class="float">.100</span>
put <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">r#{i}</span><span class="delimiter">&quot;</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">f1:c1</span><span class="delimiter">'</span></span>, i
end
flush <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_java"><a class="anchor" href="#_java"></a>75.12.2. Java</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HTableDescriptor htd = <span class="keyword">new</span> HTableDescriptor(TableName.valueOf(<span class="error"></span>test_table<span class="error"></span>));
htd.setRegionReplication(<span class="integer">2</span>);
...
admin.createTable(htd);</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can also use <code>setRegionReplication()</code> and alter table to increase, decrease the region replication for a table.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_read_api_and_usage"><a class="anchor" href="#_read_api_and_usage"></a>75.13. Read API and Usage</h3>
<div class="sect3">
<h4 id="_shell_2"><a class="anchor" href="#_shell_2"></a>75.13.1. Shell</h4>
<div class="paragraph">
<p>You can do reads in shell using a the Consistency.TIMELINE semantics as follows</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; get <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">r6</span><span class="delimiter">'</span></span>, {CONSISTENCY =&gt; <span class="string"><span class="delimiter">&quot;</span><span class="content">TIMELINE</span><span class="delimiter">&quot;</span></span>}</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can simulate a region server pausing or becoming unavailable and do a read from the secondary replica:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ kill -STOP &lt;pid or primary region server&gt;
hbase(main):001:0&gt; get 't1','r6', {CONSISTENCY =&gt; &quot;TIMELINE&quot;}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Using scans is also similar</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; scan <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>, {CONSISTENCY =&gt; <span class="string"><span class="delimiter">'</span><span class="content">TIMELINE</span><span class="delimiter">'</span></span>}</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_java_2"><a class="anchor" href="#_java_2"></a>75.13.2. Java</h4>
<div class="paragraph">
<p>You can set the consistency for Gets and Scans and do requests as follows.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Get get = <span class="keyword">new</span> Get(row);
get.setConsistency(Consistency.TIMELINE);
...
Result result = table.get(get);</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can also pass multiple gets:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Get get1 = <span class="keyword">new</span> Get(row);
get1.setConsistency(Consistency.TIMELINE);
...
ArrayList&lt;Get&gt; gets = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;Get&gt;();
gets.add(get1);
...
Result<span class="type">[]</span> results = table.get(gets);</code></pre>
</div>
</div>
<div class="paragraph">
<p>And Scans:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Scan scan = <span class="keyword">new</span> Scan();
scan.setConsistency(Consistency.TIMELINE);
...
ResultScanner scanner = table.getScanner(scan);</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can inspect whether the results are coming from primary region or not by calling the <code>Result.isStale()</code> method:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Result</span> result = table.get(get);
<span class="keyword">if</span> (result.isStale()) {
...
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_resources"><a class="anchor" href="#_resources"></a>75.14. Resources</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>More information about the design and implementation can be found at the jira issue: <a href="https://issues.apache.org/jira/browse/HBASE-10070">HBASE-10070</a></p>
</li>
<li>
<p>HBaseCon 2014 talk: <a href="https://hbase.apache.org/www.hbasecon.com/#2014-PresentationsRecordings">HBase Read High Availability Using Timeline-Consistent Region Replicas</a> also contains some details and <a href="http://www.slideshare.net/enissoz/hbase-high-availability-for-reads-with-time">slides</a>.</p>
</li>
</ol>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase_mob"><a class="anchor" href="#hbase_mob"></a>76. Storing Medium-sized Objects (MOB)</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Data comes in many sizes, and saving all of your data in HBase, including binary
data such as images and documents, is ideal. While HBase can technically handle
binary objects with cells that are larger than 100 KB in size, HBase&#8217;s normal
read and write paths are optimized for values smaller than 100KB in size. When
HBase deals with large numbers of objects over this threshold, referred to here
as medium objects, or MOBs, performance is degraded due to write amplification
caused by splits and compactions. When using MOBs, ideally your objects will be between
100KB and 10MB (see the <a href="#faq">FAQ</a>). HBase <strong><strong>FIX_VERSION_NUMBER</strong></strong> adds support
for better managing large numbers of MOBs while maintaining performance,
consistency, and low operational overhead. MOB support is provided by the work
done in <a href="https://issues.apache.org/jira/browse/HBASE-11339">HBASE-11339</a>. To
take advantage of MOB, you need to use <a href="#hfilev3">HFile version 3</a>. Optionally,
configure the MOB file reader&#8217;s cache settings for each RegionServer (see
<a href="#mob.cache.configure">Configuring the MOB Cache</a>), then configure specific columns to hold MOB data.
Client code does not need to change to take advantage of HBase MOB support. The
feature is transparent to the client.</p>
</div>
<div class="paragraph">
<p>MOB compaction</p>
</div>
<div class="paragraph">
<p>MOB data is flushed into MOB files after MemStore flush. There will be lots of MOB files
after some time. To reduce MOB file count, there is a periodic task which compacts
small MOB files into a large one (MOB compaction).</p>
</div>
<div class="sect2">
<h3 id="_configuring_columns_for_mob"><a class="anchor" href="#_configuring_columns_for_mob"></a>76.1. Configuring Columns for MOB</h3>
<div class="paragraph">
<p>You can configure columns to support MOB during table creation or alteration,
either in HBase Shell or via the Java API. The two relevant properties are the
boolean <code>IS_MOB</code> and the <code>MOB_THRESHOLD</code>, which is the number of bytes at which
an object is considered to be a MOB. Only <code>IS_MOB</code> is required. If you do not
specify the <code>MOB_THRESHOLD</code>, the default threshold value of 100 KB is used.</p>
</div>
<div class="listingblock">
<div class="title">Configure a Column for MOB Using HBase Shell</div>
<div class="content">
<pre>hbase&gt; create 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400}
hbase&gt; alter 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400}</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 26. Configure a Column for MOB Using the Java API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
HColumnDescriptor hcd = <span class="keyword">new</span> HColumnDescriptor(<span class="error"></span>f<span class="error"></span>);
hcd.setMobEnabled(<span class="predefined-constant">true</span>);
...
hcd.setMobThreshold(<span class="integer">102400L</span>);
...</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_configure_mob_compaction_policy"><a class="anchor" href="#_configure_mob_compaction_policy"></a>76.2. Configure MOB Compaction Policy</h3>
<div class="paragraph">
<p>By default, MOB files for one specific day are compacted into one large MOB file.
To reduce MOB file count more, there are other MOB Compaction policies supported.</p>
</div>
<div class="paragraph">
<p>daily policy - compact MOB Files for one day into one large MOB file (default policy)
weekly policy - compact MOB Files for one week into one large MOB file
montly policy - compact MOB Files for one month into one large MOB File</p>
</div>
<div class="listingblock">
<div class="title">Configure MOB compaction policy Using HBase Shell</div>
<div class="content">
<pre>hbase&gt; create 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'daily'}
hbase&gt; create 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'weekly'}
hbase&gt; create 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'monthly'}
hbase&gt; alter 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'daily'}
hbase&gt; alter 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'weekly'}
hbase&gt; alter 't1', {NAME =&gt; 'f1', IS_MOB =&gt; true, MOB_THRESHOLD =&gt; 102400, MOB_COMPACT_PARTITION_POLICY =&gt; 'monthly'}</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_configure_mob_compaction_mergeable_threshold"><a class="anchor" href="#_configure_mob_compaction_mergeable_threshold"></a>76.3. Configure MOB Compaction mergeable threshold</h3>
<div class="paragraph">
<p>If the size of a mob file is less than this value, it&#8217;s regarded as a small file and needs to
be merged in mob compaction. The default value is 1280MB.</p>
</div>
<div class="exampleblock">
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.mob.compaction.mergeable.threshold<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10000000000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_testing_mob"><a class="anchor" href="#_testing_mob"></a>76.4. Testing MOB</h3>
<div class="paragraph">
<p>The utility <code>org.apache.hadoop.hbase.IntegrationTestIngestWithMOB</code> is provided to assist with testing
the MOB feature. The utility is run as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ sudo -u hbase hbase org.apache.hadoop.hbase.IntegrationTestIngestWithMOB \
-threshold 1024 \
-minMobDataSize 512 \
-maxMobDataSize 5120</code></pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p><code><strong>threshold</strong></code> is the threshold at which cells are considered to be MOBs.
The default is 1 kB, expressed in bytes.</p>
</li>
<li>
<p><code><strong>minMobDataSize</strong></code> is the minimum value for the size of MOB data.
The default is 512 B, expressed in bytes.</p>
</li>
<li>
<p><code><strong>maxMobDataSize</strong></code> is the maximum value for the size of MOB data.
The default is 5 kB, expressed in bytes.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="mob.cache.configure"><a class="anchor" href="#mob.cache.configure"></a>76.5. Configuring the MOB Cache</h3>
<div class="paragraph">
<p>Because there can be a large number of MOB files at any time, as compared to the number of HFiles,
MOB files are not always kept open. The MOB file reader cache is a LRU cache which keeps the most
recently used MOB files open. To configure the MOB file reader&#8217;s cache on each RegionServer, add
the following properties to the RegionServer&#8217;s <code>hbase-site.xml</code>, customize the configuration to
suit your environment, and restart or rolling restart the RegionServer.</p>
</div>
<div class="exampleblock">
<div class="title">Example 27. Example MOB Cache Configuration</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.mob.file.cache.size<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>1000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Number of opened file handlers to cache.
A larger value will benefit reads by providing more file handlers per mob
file cache and would reduce frequent file opening and closing.
However, if this is set too high, this could lead to a &quot;too many opened file handers&quot;
The default value is 1000.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.mob.cache.evict.period<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>3600<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
The amount of time in seconds after which an unused file is evicted from the
MOB cache. The default value is 3600 seconds.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.mob.cache.evict.remain.ratio<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>0.5f<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
A multiplier (between 0.0 and 1.0), which determines how many files remain cached
after the threshold of files that remains cached after a cache eviction occurs
which is triggered by reaching the `hbase.mob.file.cache.size` threshold.
The default value is 0.5f, which means that half the files (the least-recently-used
ones) are evicted.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_mob_optimization_tasks"><a class="anchor" href="#_mob_optimization_tasks"></a>76.6. MOB Optimization Tasks</h3>
<div class="sect3">
<h4 id="_manually_compacting_mob_files"><a class="anchor" href="#_manually_compacting_mob_files"></a>76.6.1. Manually Compacting MOB Files</h4>
<div class="paragraph">
<p>To manually compact MOB files, rather than waiting for the
<a href="#mob.cache.configure">configuration</a> to trigger compaction, use the
<code>compact</code> or <code>major_compact</code> HBase shell commands. These commands
require the first argument to be the table name, and take a column
family as the second argument. and take a compaction type as the third argument.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; compact 't1', 'c1’, ‘MOB’
hbase&gt; major_compact 't1', 'c1’, ‘MOB’</pre>
</div>
</div>
<div class="paragraph">
<p>These commands are also available via <code>Admin.compact</code> and
<code>Admin.majorCompact</code> methods.</p>
</div>
</div>
</div>
</div>
</div>
<h1 id="inmemory_compaction" class="sect0"><a class="anchor" href="#inmemory_compaction"></a>In-memory Compaction</h1>
<div class="sect1">
<h2 id="imc.overview"><a class="anchor" href="#imc.overview"></a>77. Overview</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In-memory Compaction (A.K.A Accordion) is a new feature in hbase-2.0.0.
It was first introduced on the Apache HBase Blog at
<a href="https://blogs.apache.org/hbase/entry/accordion-hbase-breathes-with-in">Accordion: HBase Breathes with In-Memory Compaction</a>.
Quoting the blog:</p>
</div>
<div class="quoteblock">
<blockquote>
<div class="paragraph">
<p>Accordion reapplies the LSM principal [<em>Log-Structured-Merge Tree</em>, the design pattern upon which HBase is based] to MemStore, in order to eliminate redundancies and other overhead while the data is still in RAM. Doing so decreases the frequency of flushes to HDFS, thereby reducing the write amplification and the overall disk footprint. With less flushes, the write operations are stalled less frequently as the MemStore overflows, therefore the write performance is improved. Less data on disk also implies less pressure on the block cache, higher hit rates, and eventually better read response times. Finally, having less disk writes also means having less compaction happening in the background, i.e., less cycles are stolen from productive (read and write) work. All in all, the effect of in-memory compaction can be envisioned as a catalyst that enables the system move faster as a whole.</p>
</div>
</blockquote>
</div>
<div class="paragraph">
<p>A developer view is available at
<a href="https://blogs.apache.org/hbase/entry/accordion-developer-view-of-in">Accordion: Developer View of In-Memory Compaction</a>.</p>
</div>
<div class="paragraph">
<p>In-memory compaction works best when high data churn; overwrites or over-versions
can be eliminated while the data is still in memory. If the writes are all uniques,
it may drag write throughput (In-memory compaction costs CPU). We suggest you test
and compare before deploying to production.</p>
</div>
<div class="paragraph">
<p>In this section we describe how to enable Accordion and the available configurations.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_enabling"><a class="anchor" href="#_enabling"></a>78. Enabling</h2>
<div class="sectionbody">
<div class="paragraph">
<p>To enable in-memory compactions, set the <em>IN_MEMORY_COMPACTION</em> attribute
on per column family where you want the behavior. The <em>IN_MEMORY_COMPACTION</em>
attribute can have one of four values.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>NONE</em>: No in-memory compaction.</p>
</li>
<li>
<p><em>BASIC</em>: Basic policy enables flushing and keeps a pipeline of flushes until we trip the pipeline maximum threshold and then we flush to disk. No in-memory compaction but can help throughput as data is moved from the profligate, native ConcurrentSkipListMap data-type to more compact (and efficient) data types.</p>
</li>
<li>
<p><em>EAGER</em>: This is <em>BASIC</em> policy plus in-memory compaction of flushes (much like the on-disk compactions done to hfiles); on compaction we apply on-disk rules eliminating versions, duplicates, ttl&#8217;d cells, etc.</p>
</li>
<li>
<p><em>ADAPTIVE</em>: Adaptive compaction adapts to the workload. It applies either index compaction or data compaction based on the ratio of duplicate cells in the data. Experimental.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>To enable <em>BASIC</em> on the <em>info</em> column family in the table <em>radish</em>, disable the table and add the attribute to the <em>info</em> column family, and then reenable:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase(main):<span class="integer">002</span>:<span class="integer">0</span>&gt; disable <span class="string"><span class="delimiter">'</span><span class="content">radish</span><span class="delimiter">'</span></span>
<span class="constant">Took</span> <span class="float">0.5570</span> seconds
hbase(main):<span class="integer">003</span>:<span class="integer">0</span>&gt; alter <span class="string"><span class="delimiter">'</span><span class="content">radish</span><span class="delimiter">'</span></span>, {<span class="constant">NAME</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">info</span><span class="delimiter">'</span></span>, <span class="constant">IN_MEMORY_COMPACTION</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">BASIC</span><span class="delimiter">'</span></span>}
<span class="constant">Updating</span> all regions with the new schema...
<span class="constant">All</span> regions updated.
Done.
Took <span class="float">1.2413</span> seconds
hbase(main):<span class="integer">004</span>:<span class="integer">0</span>&gt; describe <span class="string"><span class="delimiter">'</span><span class="content">radish</span><span class="delimiter">'</span></span>
<span class="constant">Table</span> radish is <span class="constant">DISABLED</span>
radish
<span class="constant">COLUMN</span> <span class="constant">FAMILIES</span> <span class="constant">DESCRIPTION</span>
{<span class="constant">NAME</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">info</span><span class="delimiter">'</span></span>, <span class="constant">VERSIONS</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="constant">EVICT_BLOCKS_ON_CLOSE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">NEW_VERSION_BEHAVIOR</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">KEEP_DELETED_CELLS</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">FALSE</span><span class="delimiter">'</span></span>, <span class="constant">CACHE_DATA_ON_WRITE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">DATA_BLOCK_ENCODING</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">NONE</span><span class="delimiter">'</span></span>, <span class="constant">TTL</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">FOREVER</span><span class="delimiter">'</span></span>, <span class="constant">MIN_VERSIONS</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">0</span><span class="delimiter">'</span></span>, <span class="constant">REPLICATION_SCOPE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">0</span><span class="delimiter">'</span></span>, <span class="constant">BLOOMFILTER</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">ROW</span><span class="delimiter">'</span></span>, <span class="constant">CACHE_INDEX_ON_WRITE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">IN_MEMORY</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">CACHE_BLOOMS_ON_WRITE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">PREFETCH_BLOCKS_ON_OPEN</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">false</span><span class="delimiter">'</span></span>, <span class="constant">COMPRESSION</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">NONE</span><span class="delimiter">'</span></span>, <span class="constant">BLOCKCACHE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">true</span><span class="delimiter">'</span></span>, <span class="constant">BLOCKSIZE</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">65536</span><span class="delimiter">'</span></span>, <span class="constant">METADATA</span> =&gt; {
<span class="string"><span class="delimiter">'</span><span class="content">IN_MEMORY_COMPACTION</span><span class="delimiter">'</span></span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">BASIC</span><span class="delimiter">'</span></span>}}
<span class="integer">1</span> row(s)
<span class="constant">Took</span> <span class="float">0.0239</span> seconds
hbase(main):<span class="integer">005</span>:<span class="integer">0</span>&gt; enable <span class="string"><span class="delimiter">'</span><span class="content">radish</span><span class="delimiter">'</span></span>
<span class="constant">Took</span> <span class="float">0.7537</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note how the IN_MEMORY_COMPACTION attribute shows as part of the <em>METADATA</em> map.</p>
</div>
<div class="paragraph">
<p>There is also a global configuration, <em>hbase.hregion.compacting.memstore.type</em> which you can set in your <em>hbase-site.xml</em> file. Use it to set the
default on creation of a new table (On creation of a column family Store, we look first to the column family configuration looking for the
<em>IN_MEMORY_COMPACTION</em> setting, and if none, we then consult the <em>hbase.hregion.compacting.memstore.type</em> value using its content; default is
<em>BASIC</em>).</p>
</div>
<div class="paragraph">
<p>By default, new hbase system tables will have <em>BASIC</em> in-memory compaction set. To specify otherwise,
on new table-creation, set <em>hbase.hregion.compacting.memstore.type</em> to <em>NONE</em> (Note, setting this value
post-creation of system tables will not have a retroactive effect; you will have to alter your tables
to set the in-memory attribute to <em>NONE</em>).</p>
</div>
<div class="paragraph">
<p>When an in-memory flush happens is calculated by dividing the configured region flush size (Set in the table descriptor
or read from <em>hbase.hregion.memstore.flush.size</em>) by the number of column families and then multiplying by
<em>hbase.memstore.inmemoryflush.threshold.factor</em>. Default is 0.014.</p>
</div>
<div class="paragraph">
<p>The number of flushes carried by the pipeline is monitored so as to fit within the bounds of memstore sizing
but you can also set a maximum on the number of flushes total by setting
<em>hbase.hregion.compacting.pipeline.segments.limit</em>. Default is 2.</p>
</div>
<div class="paragraph">
<p>When a column family Store is created, it says what memstore type is in effect. As of this writing
there is the old-school <em>DefaultMemStore</em> which fills a <em>ConcurrentSkipListMap</em> and then flushes
to disk or the new <em>CompactingMemStore</em> that is the implementation that provides this new
in-memory compactions facility. Here is a log-line from a RegionServer that shows a column
family Store named <em>family</em> configured to use a <em>CompactingMemStore</em>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Note how the IN_MEMORY_COMPACTION attribute shows as part of the _METADATA_ map.
2018-03-30 11:02:24,466 INFO [Time-limited test] regionserver.HStore(325): Store=family, memstore type=CompactingMemStore, storagePolicy=HOT, verifyBulkLoads=false, parallelPutCountPrintThreshold=10</pre>
</div>
</div>
<div class="paragraph">
<p>Enable TRACE-level logging on the CompactingMemStore class (<em>org.apache.hadoop.hbase.regionserver.CompactingMemStore</em>) to see detail on its operation.</p>
</div>
</div>
</div>
<h1 id="offheap_read_write" class="sect0"><a class="anchor" href="#offheap_read_write"></a>RegionServer Offheap Read/Write Path</h1>
<div class="sect1">
<h2 id="regionserver.offheap.overview"><a class="anchor" href="#regionserver.offheap.overview"></a>79. Overview</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For reducing the Java GC impact to P99/P999 RPC latency, HBase 2.x has made the offheap read and write path. The cells are
allocated from JVM offheap memory area, which won’t be garbage collected by JVM and need to be deallocated explicitly by
upstream callers. In the write path, the request packet received from client will be allocated offheap and retained
until those cells are successfully written to the WAL and Memstore. The memory data structure in Memstore does
not directly store the cell memory, but reference to cells which are encoded in multiple chunks in MSLAB, this is easier
to manage the offheap memory. Similarly, in the read path, we’ll try to read the cache firstly, if the cache
misses, go to the HFile and read the corresponding block. The workflow: from reading blocks to sending cells to
client, it&#8217;s basically not involved in on-heap memory allocations.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/offheap-overview.png" alt="offheap overview">
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="regionserver.offheap.readpath"><a class="anchor" href="#regionserver.offheap.readpath"></a>80. Offheap read-path</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In HBase-2.0.0, <a href="https://issues.apache.org/jira/browse/HBASE-11425">HBASE-11425</a> changed the HBase read path so it
could hold the read-data off-heap (from BucketCache) avoiding copying of cached data on to the java heap.
This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path has a performance
that is similar/better to that of the on-heap LRU cache. This feature is available since HBase 2.0.0.
If the BucketCache is in <code>file</code> mode, fetching will always be slower compared to the native on-heap LruBlockCache.
Refer to below blogs for more details and test results on off heaped read path
<a href="https://blogs.apache.org/hbase/entry/offheaping_the_read_path_in">Offheaping the Read Path in Apache HBase: Part 1 of 2</a>
and <a href="https://blogs.apache.org/hbase/entry/offheap-read-path-in-production">Offheap Read-Path in Production - The Alibaba story</a></p>
</div>
<div class="paragraph">
<p>For an end-to-end off-heaped read-path, first of all there should be an off-heap backed <a href="#offheap.blockcache">BucketCache</a>. Configure 'hbase.bucketcache.ioengine' to off-heap in
<em>hbase-site.xml</em>. Also specify the total capacity of the BucketCache using <code>hbase.bucketcache.size</code> config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
<em>hbase-env.sh</em>. This is how we specify the max possible off-heap memory allocation for the RegionServer java process.
This should be bigger than the off-heap BC size. Please keep in mind that there is no default for <code>hbase.bucketcache.ioengine</code>
which means the BC is turned OFF by default (See <a href="#direct.memory">Direct Memory Usage In HBase</a>).</p>
</div>
<div class="paragraph">
<p>Next thing to tune is the ByteBuffer pool on the RPC server side:</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
the config keys which start with prefix <code>hbase.ipc.server.reservoir</code> are deprecated in HBase3.x. If you are still
in HBase2.x, then just use the old config keys. otherwise if in HBase3.x, please use the new config keys.
(See <a href="#regionserver.read.hdfs.block.offheap">deprecated and new configs in HBase3.x</a>)
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>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.
<code>hbase.ipc.server.reservoir.enabled</code> can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off heap ByteBuffers
and pool them. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and make a result cell block. This can impact the GC on a highly read loaded server.
The user can tune this pool with respect to how many buffers are in the pool and what should be the size of each ByteBuffer.
Use the config <code>hbase.ipc.server.reservoir.initial.buffer.size</code> to tune each of the buffer sizes. Default is 64 KB for HBase2.x, while it will be changed to 65KB by default for HBase3.x
(see <a href="https://issues.apache.org/jira/browse/HBASE-22532">HBASE-22532</a>)</p>
</div>
<div class="paragraph">
<p>When the result size is larger than one ByteBuffer size, the server will try to grab more than one ByteBuffer 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.</p>
</div>
<div class="paragraph">
<p>The maximum number of ByteBuffers in the pool can be tuned using the config <code>hbase.ipc.server.reservoir.initial.max</code>.
Its value defaults to 64 * region server handlers configured (See the config <code>hbase.regionserver.handler.count</code>). 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.</p>
</div>
<div class="paragraph">
<p>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 in HBase2.x:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value <span class="keyword">for</span> <span class="string"><span class="delimiter">'</span><span class="content">hbase.ipc.server.reservoir.initial.max</span><span class="delimiter">'</span></span> ?</code></pre>
</div>
</div>
<div class="paragraph">
<p>Or the following log message in HBase3.x:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value <span class="keyword">for</span> <span class="string"><span class="delimiter">'</span><span class="content">hbase.server.allocator.max.buffer.count</span><span class="delimiter">'</span></span> ?</code></pre>
</div>
</div>
<div class="paragraph">
<p>The setting for <em>HBASE_OFFHEAPSIZE</em> in <em>hbase-env.sh</em> 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.</p>
</div>
<div class="paragraph">
<p>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.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="regionserver.read.hdfs.block.offheap"><a class="anchor" href="#regionserver.read.hdfs.block.offheap"></a>81. Read block from HDFS to offheap directly</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In HBase-2.x, the RegionServer will still read block from HDFS to a temporary heap ByteBuffer and then flush to BucketCache&#8217;s
IOEngine asynchronously, finally it will be an offheap one. We can still observe much GC pressure when cache hit ratio
is not very high (such as cacheHitRatio ~ 60% ), so in <a href="https://issues.apache.org/jira/browse/HBASE-21879">HBASE-21879</a>
we redesigned the read path and made the HDFS block reading be offheap now. This feature will be available in HBASE-3.0.0.</p>
</div>
<div class="paragraph">
<p>For more details about the design and performance improvement, please see the <a href="https://docs.google.com/document/d/1xSy9axGxafoH-Qc17zbD2Bd&#8212;&#8203;rWjjI00xTWQZ8ZwI_E/edit?usp=sharing">document</a>.
Here we will share some best practice about the performance tuning:</p>
</div>
<div class="paragraph">
<p>Firstly, we introduced several configurations about the ByteBuffAllocator (which was abstracted to manage the memory application or release):</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><code>hbase.server.allocator.pool.enabled</code>: means whether the region server will use the pooled offheap ByteBuffer allocator. Its default
value is true. In HBase2.x, we still use the deprecated <code>hbase.ipc.server.reservoir.enabled</code> config while we&#8217;ll use the new
one in HBase3.x.</p>
</li>
<li>
<p><code>hbase.server.allocator.minimal.allocate.size</code>: If the desired byte size is not less than this one, then it will
be allocated as a pooled offheap ByteBuff, otherwise it will be allocated from heap directly because it
is too wasting to allocate from pool with fixed-size ByteBuffers, default value is <code>hbase.server.allocator.buffer.size/6</code>.</p>
</li>
<li>
<p><code>hbase.server.allocator.max.buffer.count</code>: The ByteBuffAllocator will have many fixed-size ByteBuffers inside which
are composited as a pool, this config indicate how many buffers are there in the pool. Its default value will be 2MB * 2 * hbase.regionserver.handler.count / 65KB,
the default hbase.regionserver.handler.count is 30, then its value will be 1890.</p>
</li>
<li>
<p><code>hbase.server.allocator.buffer.size</code>: The byte size of each ByteBuffer, default value is 66560 (65KB), here we choose 65KB instead of 64KB
because of <a href="https://issues.apache.org/jira/browse/HBASE-22532">HBASE-22532</a>.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The three config keys: <code>hbase.ipc.server.reservoir.enabled</code>, <code>hbase.ipc.server.reservoir.initial.buffer.size</code> and <code>hbase.ipc.server.reservoir.initial.max</code> are introduced in HBase2.x. while in HBase3.x
they are deprecated now, instead please use the new config keys: <code>hbase.server.allocator.pool.enabled</code>, <code>hbase.server.allocator.buffer.size</code> and <code>hbase.server.allocator.max.buffer.count</code>.</p>
</div>
<div class="paragraph">
<p>If you still use the deprecated three config keys in HBase3.0.0, you will get a WARN log message like:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">The config keys hbase.ipc.server.reservoir.initial.buffer.size and hbase.ipc.server.reservoir.initial.max are deprecated now, instead please use hbase.server.allocator.buffer.size and hbase.server.allocator.max.buffer.count. In future release we will remove the two deprecated configs.</code></pre>
</div>
</div>
<div class="paragraph">
<p>Second, we have some suggestions about the performance:</p>
</div>
<div class="paragraph">
<div class="title">Please make sure that there are enough pooled DirectByteBuffer in your ByteBuffAllocator.</div>
<p>The ByteBuffAllocator will allocate ByteBuffer from DirectByteBuffer pool firstly, if there’s no available ByteBuffer
from the pool, then it will just allocate the ByteBuffers from heap, then the GC pressures will increase again.</p>
</div>
<div class="paragraph">
<p>By default, we will pre-allocate 4MB for each RPC handlers ( The handler count is determined by the config:
<code>hbase.regionserver.handler.count</code>, it has the default value 30) . That’s to say, if your <code>hbase.server.allocator.buffer.size</code>
is 65KB, then your pool will have 2MB * 2 / 65KB * 30 = 945 DirectByteBuffer. If you have some large scan and have a big caching,
say you may have a rpc response whose bytes size is greater than 2MB (another 2MB for receiving rpc request), then it will
be better to increase the <code>hbase.server.allocator.max.buffer.count</code>.</p>
</div>
<div class="paragraph">
<p>The RegionServer web UI also has the statistic about ByteBuffAllocator:</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/bytebuff-allocator-stats.png" alt="bytebuff allocator stats">
</div>
</div>
<div class="paragraph">
<p>If the following condition meet, you may need to increase your max buffer.count:</p>
</div>
<div class="paragraph">
<p>heapAllocationRatio &gt;= hbase.server.allocator.minimal.allocate.size / hbase.server.allocator.buffer.size * 100%</p>
</div>
<div class="paragraph">
<div class="title">Please make sure the buffer size is greater than your block size.</div>
<p>We have the default block size=64KB, so almost all of the data block have a block size: 64KB + delta, whose delta is
very small, depends on the size of last KeyValue. If we use the default <code>hbase.server.allocator.buffer.size</code>=64KB,
then each block will be allocated as two ByteBuffers: one 64KB DirectByteBuffer and one HeapByteBuffer with delta bytes,
the HeapByteBuffer will increase the GC pressure. Ideally, we should let the data block to be allocated as one ByteBuffer,
it has simpler data structure, faster access speed, less heap usage. On the other hand, If the blocks are composited by multiple ByteBuffers,
so we have to validate the checksum by an temporary heap copying (see <a href="https://issues.apache.org/jira/browse/HBASE-21917">HBASE-21917</a>), while if it’s a single ByteBuffer,
we can speed the checksum by calling the hadoop' checksum in native lib, it&#8217;s more faster.</p>
</div>
<div class="paragraph">
<p>Please also see: <a href="https://issues.apache.org/jira/browse/HBASE-22483">HBASE-22483</a></p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="regionserver.offheap.writepath"><a class="anchor" href="#regionserver.offheap.writepath"></a>82. Offheap write-path</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In HBase 2.0.0, <a href="https://issues.apache.org/jira/browse/HBASE-15179">HBASE-15179</a> made the HBase write path to work off-heap. By default, the MemStores use
MSLAB to avoid memory fragmentation. It creates bigger fixed sized chunks and memstore cell&#8217;s data will get copied into these chunks. These chunks can be pooled
also and from 2.0.0 the MSLAB (MemStore-Local Allocation Buffer) pool is by default ON. Write off-heaping makes use of the MSLAB pool. It creates MSLAB chunks
as Direct ByteBuffers and pools them. HBase defaults to using no off-heap memory for MSLAB which means that cells are copied to heap chunk in MSLAB by default
rather than off-heap chunk.</p>
</div>
<div class="paragraph">
<p><code>hbase.regionserver.offheap.global.memstore.size</code> is the configuration key which controls the amount of off-heap data whose value is the number of megabytes
of off-heap memory that should be by MSLAB (e.g. <code>25</code> would result in 25MB of off-heap). Be sure to increase <code>HBASE_OFFHEAPSIZE</code> which will set the JVM&#8217;s
MaxDirectMemorySize property. Its default value is 0, means MSLAB use heap chunks.</p>
</div>
<div class="paragraph">
<p><code>hbase.hregion.memstore.mslab.chunksize</code> controls the size of each off-heap chunk, defaulting to <code>2097152</code> (2MB).</p>
</div>
<div class="paragraph">
<p>When a Cell is added to a MemStore, the bytes for that Cell are copied into these off-heap buffers (if set the <code>hbase.regionserver.offheap.global.memstore.size</code> to non-zero)
and a Cell POJO will refer to this memory area. This can greatly reduce the on-heap occupancy of the MemStores and reduce the total heap utilization for RegionServers
in a write-heavy workload. On-heap and off-heap memory utiliazation are tracked at multiple levels to implement low level and high level memory management.
The decision to flush a MemStore considers both the on-heap and off-heap usage of that MemStore. At the Region level, the sum of the on-heap and off-heap usages and
compares them against the region flush size (128MB, by default). Globally, on-heap size occupancy of all memstores are tracked as well as off-heap size. When any of
these sizes breaches the lower mark (<code>hbase.regionserver.global.memstore.size.lower.limit</code>) or the maximum size <code>hbase.regionserver.global.memstore.size</code>), all
regions are selected for forced flushes.</p>
</div>
</div>
</div>
<h1 id="backuprestore" class="sect0"><a class="anchor" href="#backuprestore"></a>Backup and Restore</h1>
<div class="sect1">
<h2 id="br.overview"><a class="anchor" href="#br.overview"></a>83. Overview</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Backup and restore is a standard operation provided by many databases. An effective backup and restore
strategy helps ensure that users can recover data in case of unexpected failures. The HBase backup and restore
feature helps ensure that enterprises using HBase as a canonical data repository can recover from catastrophic
failures. Another important feature is the ability to restore the database to a particular
point-in-time, commonly referred to as a snapshot.</p>
</div>
<div class="paragraph">
<p>The HBase backup and restore feature provides the ability to create full backups and incremental backups on
tables in an HBase cluster. The full backup is the foundation on which incremental backups are applied
to build iterative snapshots. Incremental backups can be run on a schedule to capture changes over time,
for example by using a Cron task. Incremental backups are more cost-effective than full backups because they only capture
the changes since the last backup and they also enable administrators to restore the database to any prior incremental backup. Furthermore, the
utilities also enable table-level data backup-and-recovery if you do not want to restore the entire dataset
of the backup.</p>
</div>
<div class="paragraph">
<p>The backup and restore feature supplements the HBase Replication feature. While HBase replication is ideal for
creating "hot" copies of the data (where the replicated data is immediately available for query), the backup and
restore feature is ideal for creating "cold" copies of data (where a manual step must be taken to restore the system).
Previously, users only had the ability to create full backups via the ExportSnapshot functionality. The incremental
backup implementation is the novel improvement over the previous "art" provided by ExportSnapshot.</p>
</div>
<div class="paragraph">
<p>The backup and restore feature uses DistCp to transfer files between clusters .
<a href="https://issues.apache.org/jira/browse/HADOOP-15850">HADOOP-15850</a> fixes a bug where CopyCommitter#concatFileChunks
unconditionally tried to concatenate the files being DistCp&#8217;ed to target cluster (though the files are
independent) . Without the fix from
<a href="https://issues.apache.org/jira/browse/HADOOP-15850">HADOOP-15850</a> , the transfer would fail.
So the backup and restore feature need hadoop version as below</p>
</div>
<div class="ulist">
<ul>
<li>
<p>2.7.x</p>
</li>
<li>
<p>2.8.x</p>
</li>
<li>
<p>2.9.2+</p>
</li>
<li>
<p>2.10.0+</p>
</li>
<li>
<p>3.0.4+</p>
</li>
<li>
<p>3.1.2+</p>
</li>
<li>
<p>3.2.0+</p>
</li>
<li>
<p>3.3.0+</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.terminology"><a class="anchor" href="#br.terminology"></a>84. Terminology</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The backup and restore feature introduces new terminology which can be used to understand how control flows through the
system.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>A backup</em>: A logical unit of data and metadata which can restore a table to its state at a specific point in time.</p>
</li>
<li>
<p><em>Full backup</em>: a type of backup which wholly encapsulates the contents of the table at a point in time.</p>
</li>
<li>
<p><em>Incremental backup</em>: a type of backup which contains the changes in a table since a full backup.</p>
</li>
<li>
<p><em>Backup set</em>: A user-defined name which references one or more tables over which a backup can be executed.</p>
</li>
<li>
<p><em>Backup ID</em>: A unique names which identifies one backup from the rest, e.g. <code>backupId_1467823988425</code></p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.planning"><a class="anchor" href="#br.planning"></a>85. Planning</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are some common strategies which can be used to implement backup and restore in your environment. The following section
shows how these strategies are implemented and identifies potential tradeoffs with each.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
This backup and restore tools has not been tested on Transparent Data Encryption (TDE) enabled HDFS clusters.
This is related to the open issue <a href="https://issues.apache.org/jira/browse/HBASE-16178">HBASE-16178</a>.
</td>
</tr>
</table>
</div>
<div class="sect2">
<h3 id="br.intracluster.backup"><a class="anchor" href="#br.intracluster.backup"></a>85.1. Backup within a cluster</h3>
<div class="paragraph">
<p>This strategy stores the backups on the same cluster as where the backup was taken. This approach is only appropriate for testing
as it does not provide any additional safety on top of what the software itself already provides.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/backup-intra-cluster.png" alt="backup intra cluster">
</div>
<div class="title">Figure 5. Intra-Cluster Backup</div>
</div>
</div>
<div class="sect2">
<h3 id="br.dedicated.cluster.backup"><a class="anchor" href="#br.dedicated.cluster.backup"></a>85.2. Backup using a dedicated cluster</h3>
<div class="paragraph">
<p>This strategy provides greater fault tolerance and provides a path towards disaster recovery. In this setting, you will
store the backup on a separate HDFS cluster by supplying the backup destination cluster’s HDFS URL to the backup utility.
You should consider backing up to a different physical location, such as a different data center.</p>
</div>
<div class="paragraph">
<p>Typically, a backup-dedicated HDFS cluster uses a more economical hardware profile to save money.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/backup-dedicated-cluster.png" alt="backup dedicated cluster">
</div>
<div class="title">Figure 6. Dedicated HDFS Cluster Backup</div>
</div>
</div>
<div class="sect2">
<h3 id="br.cloud.or.vendor.backup"><a class="anchor" href="#br.cloud.or.vendor.backup"></a>85.3. Backup to the Cloud or a storage vendor appliance</h3>
<div class="paragraph">
<p>Another approach to safeguarding HBase incremental backups is to store the data on provisioned, secure servers that belong
to third-party vendors and that are located off-site. The vendor can be a public cloud provider or a storage vendor who uses
a Hadoop-compatible file system, such as S3 and other HDFS-compatible destinations.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/backup-cloud-appliance.png" alt="backup cloud appliance">
</div>
<div class="title">Figure 7. Backup to Cloud or Vendor Storage Solutions</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The HBase backup utility does not support backup to multiple destinations. A workaround is to manually create copies
of the backup files from HDFS or S3.
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.initial.setup"><a class="anchor" href="#br.initial.setup"></a>86. First-time configuration steps</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This section contains the necessary configuration changes that must be made in order to use the backup and restore feature.
As this feature makes significant use of YARN&#8217;s MapReduce framework to parallelize these I/O heavy operations, configuration
changes extend outside of just <code>hbase-site.xml</code>.</p>
</div>
<div class="sect2">
<h3 id="_allow_the_hbase_system_user_in_yarn"><a class="anchor" href="#_allow_the_hbase_system_user_in_yarn"></a>86.1. Allow the "hbase" system user in YARN</h3>
<div class="paragraph">
<p>The YARN <strong>container-executor.cfg</strong> configuration file must have the following property setting: <em>allowed.system.users=hbase</em>. No spaces
are allowed in entries of this configuration file.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
Skipping this step will result in runtime errors when executing the first backup tasks.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p><strong>Example of a valid container-executor.cfg file for backup and restore:</strong></p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">yarn.nodemanager.log-dirs=/var/log/hadoop/mapred
yarn.nodemanager.linux-container-executor.group=yarn
banned.users=hdfs,yarn,mapred,bin
allowed.system.users=hbase
min.user.id=<span class="integer">500</span></code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_specific_changes"><a class="anchor" href="#_hbase_specific_changes"></a>86.2. HBase specific changes</h3>
<div class="paragraph">
<p>Add the following properties to hbase-site.xml and restart HBase if it is already running.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The ",&#8230;&#8203;" is an ellipsis meant to imply that this is a comma-separated list of values, not literal text which should be added to hbase-site.xml.
</td>
</tr>
</table>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">&lt;property&gt;
&lt;name&gt;hbase.backup.enable&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">true</span>&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.master.logcleaner.plugins&lt;/name&gt;
&lt;value&gt;org.apache.hadoop.hbase.backup.master.BackupLogCleaner,...&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.procedure.master.classes&lt;/name&gt;
&lt;value&gt;org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager,...&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.procedure.regionserver.classes&lt;/name&gt;
&lt;value&gt;org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager,...&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.coprocessor.region.classes&lt;/name&gt;
&lt;value&gt;org.apache.hadoop.hbase.backup.BackupObserver,...&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.master.hfilecleaner.plugins&lt;/name&gt;
&lt;value&gt;org.apache.hadoop.hbase.backup.BackupHFileCleaner,...&lt;/value&gt;
&lt;/property&gt;</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_backup_and_restore_commands"><a class="anchor" href="#_backup_and_restore_commands"></a>87. Backup and Restore commands</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This covers the command-line utilities that administrators would run to create, restore, and merge backups. Tools to
inspect details on specific backup sessions is covered in the next section, <a href="#br.administration">Administration of Backup Images</a>.</p>
</div>
<div class="paragraph">
<p>Run the command <code>hbase backup help &lt;command&gt;</code> to access the online help that provides basic information about a command
and its options. The below information is captured in this help message for each command.</p>
</div>
<div class="sect2">
<h3 id="br.creating.complete.backup"><a class="anchor" href="#br.creating.complete.backup"></a>87.1. Creating a Backup Image</h3>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="paragraph">
<p>For HBase clusters also using Apache Phoenix: include the SQL system catalog tables in the backup. In the event that you
need to restore the HBase backup, access to the system catalog tables enable you to resume Phoenix interoperability with the
restored data.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The first step in running the backup and restore utilities is to perform a full backup and to store the data in a separate image
from the source. At a minimum, you must do this to get a baseline before you can rely on incremental backups.</p>
</div>
<div class="paragraph">
<p>Run the following command as HBase superuser:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase backup create &lt;type&gt; &lt;backup_path&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>After the command finishes running, the console prints a SUCCESS or FAILURE status message. The SUCCESS message includes a <em>backup</em> ID.
The backup ID is the Unix time (also known as Epoch time) that the HBase master received the backup request from the client.</p>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
<div class="paragraph">
<p>Record the backup ID that appears at the end of a successful backup. In case the source cluster fails and you need to recover the
dataset with a restore operation, having the backup ID readily available can save time.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="br.create.positional.cli.arguments"><a class="anchor" href="#br.create.positional.cli.arguments"></a>87.1.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>type</em></dt>
<dd>
<p>The type of backup to execute: <em>full</em> or <em>incremental</em>. As a reminder, an <em>incremental</em> backup requires a <em>full</em> backup to
already exist.</p>
</dd>
<dt class="hdlist1"><em>backup_path</em></dt>
<dd>
<p>The <em>backup_path</em> argument specifies the full filesystem URI of where to store the backup image. Valid prefixes are
<em>hdfs:</em>, <em>webhdfs:</em>, <em>s3a:</em> or other compatible Hadoop File System implementations.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.create.named.cli.arguments"><a class="anchor" href="#br.create.named.cli.arguments"></a>87.1.2. Named Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>-t &lt;table_name[,table_name]&gt;</em></dt>
<dd>
<p>A comma-separated list of tables to back up. If no tables are specified, all tables are backed up. No regular-expression or
wildcard support is present; all table names must be explicitly listed. See <a href="#br.using.backup.sets">Backup Sets</a> for more
information about peforming operations on collections of tables. Mutually exclusive with the <em>-s</em> option; one of these
named options are required.</p>
</dd>
<dt class="hdlist1"><em>-s &lt;backup_set_name&gt;</em></dt>
<dd>
<p>Identify tables to backup based on a backup set. See <a href="#br.using.backup.sets">Using Backup Sets</a> for the purpose and usage
of backup sets. Mutually exclusive with the <em>-t</em> option.</p>
</dd>
<dt class="hdlist1"><em>-w &lt;number_workers&gt;</em></dt>
<dd>
<p>(Optional) Specifies the number of parallel workers to copy data to backup destination. Backups are currently executed by MapReduce jobs
so this value corresponds to the number of Mappers that will be spawned by the job.</p>
</dd>
<dt class="hdlist1"><em>-b &lt;bandwidth_per_worker&gt;</em></dt>
<dd>
<p>(Optional) Specifies the bandwidth of each worker in MB per second.</p>
</dd>
<dt class="hdlist1"><em>-d</em></dt>
<dd>
<p>(Optional) Enables "DEBUG" mode which prints additional logging about the backup creation.</p>
</dd>
<dt class="hdlist1"><em>-q &lt;name&gt;</em></dt>
<dd>
<p>(Optional) Allows specification of the name of a YARN queue which the MapReduce job to create the backup should be executed in. This option
is useful to prevent backup tasks from stealing resources away from other MapReduce jobs of high importance.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.usage.examples"><a class="anchor" href="#br.usage.examples"></a>87.1.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup create full hdfs:<span class="comment">//host5:8020/data/backup -t SALES2,SALES3 -w 3</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>This command creates a full backup image of two tables, SALES2 and SALES3, in the HDFS instance who NameNode is host5:8020
in the path <em>/data/backup</em>. The <em>-w</em> option specifies that no more than three parallel works complete the operation.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.restoring.backup"><a class="anchor" href="#br.restoring.backup"></a>87.2. Restoring a Backup Image</h3>
<div class="paragraph">
<p>Run the following command as an HBase superuser. You can only restore a backup on a running HBase cluster because the data must be
redistributed the RegionServers for the operation to complete successfully.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase restore &lt;backup_path&gt; &lt;backup_id&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.restore.positional.args"><a class="anchor" href="#br.restore.positional.args"></a>87.2.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_path</em></dt>
<dd>
<p>The <em>backup_path</em> argument specifies the full filesystem URI of where to store the backup image. Valid prefixes are
<em>hdfs:</em>, <em>webhdfs:</em>, <em>s3a:</em> or other compatible Hadoop File System implementations.</p>
</dd>
<dt class="hdlist1"><em>backup_id</em></dt>
<dd>
<p>The backup ID that uniquely identifies the backup image to be restored.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.restore.named.args"><a class="anchor" href="#br.restore.named.args"></a>87.2.2. Named Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>-t &lt;table_name[,table_name]&gt;</em></dt>
<dd>
<p>A comma-separated list of tables to restore. See <a href="#br.using.backup.sets">Backup Sets</a> for more
information about peforming operations on collections of tables. Mutually exclusive with the <em>-s</em> option; one of these
named options are required.</p>
</dd>
<dt class="hdlist1"><em>-s &lt;backup_set_name&gt;</em></dt>
<dd>
<p>Identify tables to backup based on a backup set. See <a href="#br.using.backup.sets">Using Backup Sets</a> for the purpose and usage
of backup sets. Mutually exclusive with the <em>-t</em> option.</p>
</dd>
<dt class="hdlist1"><em>-q &lt;name&gt;</em></dt>
<dd>
<p>(Optional) Allows specification of the name of a YARN queue which the MapReduce job to create the backup should be executed in. This option
is useful to prevent backup tasks from stealing resources away from other MapReduce jobs of high importance.</p>
</dd>
<dt class="hdlist1"><em>-c</em></dt>
<dd>
<p>(Optional) Perform a dry-run of the restore. The actions are checked, but not executed.</p>
</dd>
<dt class="hdlist1"><em>-m &lt;target_tables&gt;</em></dt>
<dd>
<p>(Optional) A comma-separated list of tables to restore into. If this option is not provided, the original table name is used. When
this option is provided, there must be an equal number of entries provided in the <code>-t</code> option.</p>
</dd>
<dt class="hdlist1"><em>-o</em></dt>
<dd>
<p>(Optional) Overwrites the target table for the restore if the table already exists.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.restore.usage"><a class="anchor" href="#br.restore.usage"></a>87.2.3. Example of Usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase restore /tmp/backup_incremental backupId_1467823988425 -t mytable1,mytable2</code></pre>
</div>
</div>
<div class="paragraph">
<p>This command restores two tables of an incremental backup image. In this example:
<code>/tmp/backup_incremental</code> is the path to the directory containing the backup image.
<code>backupId_1467823988425</code> is the backup ID.
<code>mytable1</code> and <code>mytable2</code> are the names of tables in the backup image to be restored.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.merge.backup"><a class="anchor" href="#br.merge.backup"></a>87.3. Merging Incremental Backup Images</h3>
<div class="paragraph">
<p>This command can be used to merge two or more incremental backup images into a single incremental
backup image. This can be used to consolidate multiple, small incremental backup images into a single
larger incremental backup image. This command could be used to merge hourly incremental backups
into a daily incremental backup image, or daily incremental backups into a weekly incremental backup.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup merge &lt;backup_ids&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.merge.backup.positional.cli.arguments"><a class="anchor" href="#br.merge.backup.positional.cli.arguments"></a>87.3.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_ids</em></dt>
<dd>
<p>A comma-separated list of incremental backup image IDs that are to be combined into a single image.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.merge.backup.named.cli.arguments"><a class="anchor" href="#br.merge.backup.named.cli.arguments"></a>87.3.2. Named Command-Line Arguments</h4>
<div class="paragraph">
<p>None.</p>
</div>
</div>
<div class="sect3">
<h4 id="br.merge.backup.example"><a class="anchor" href="#br.merge.backup.example"></a>87.3.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup merge backupId_1467823988425,backupId_1467827588425</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.using.backup.sets"><a class="anchor" href="#br.using.backup.sets"></a>87.4. Using Backup Sets</h3>
<div class="paragraph">
<p>Backup sets can ease the administration of HBase data backups and restores by reducing the amount of repetitive input
of table names. You can group tables into a named backup set with the <code>hbase backup set add</code> command. You can then use
the <code>-set</code> option to invoke the name of a backup set in the <code>hbase backup create</code> or <code>hbase restore</code> rather than list
individually every table in the group. You can have multiple backup sets.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Note the differentiation between the <code>hbase backup set add</code> command and the <em>-set</em> option. The <code>hbase backup set add</code>
command must be run before using the <code>-set</code> option in a different command because backup sets must be named and defined
before using backup sets as a shortcut.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>If you run the <code>hbase backup set add</code> command and specify a backup set name that does not yet exist on your system, a new set
is created. If you run the command with the name of an existing backup set name, then the tables that you specify are added
to the set.</p>
</div>
<div class="paragraph">
<p>In this command, the backup set name is case-sensitive.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The metadata of backup sets are stored within HBase. If you do not have access to the original HBase cluster with the
backup set metadata, then you must specify individual table names to restore the data.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>To create a backup set, run the following command as the HBase superuser:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup set &lt;subcommand&gt; &lt;backup_set_name&gt; &lt;tables&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.set.subcommands"><a class="anchor" href="#br.set.subcommands"></a>87.4.1. Backup Set Subcommands</h4>
<div class="paragraph">
<p>The following list details subcommands of the hbase backup set command.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
You must enter one (and no more than one) of the following subcommands after hbase backup set to complete an operation.
Also, the backup set name is case-sensitive in the command-line utility.
</td>
</tr>
</table>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>add</em></dt>
<dd>
<p>Adds table[s] to a backup set. Specify a <em>backup_set_name</em> value after this argument to create a backup set.</p>
</dd>
<dt class="hdlist1"><em>remove</em></dt>
<dd>
<p>Removes tables from the set. Specify the tables to remove in the tables argument.</p>
</dd>
<dt class="hdlist1"><em>list</em></dt>
<dd>
<p>Lists all backup sets.</p>
</dd>
<dt class="hdlist1"><em>describe</em></dt>
<dd>
<p>Displays a description of a backup set. The information includes whether the set has full
or incremental backups, start and end times of the backups, and a list of the tables in the set. This subcommand must precede
a valid value for the <em>backup_set_name</em> value.</p>
</dd>
<dt class="hdlist1"><em>delete</em></dt>
<dd>
<p>Deletes a backup set. Enter the value for the <em>backup_set_name</em> option directly after the <code>hbase backup set delete</code> command.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.set.positional.cli.arguments"><a class="anchor" href="#br.set.positional.cli.arguments"></a>87.4.2. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_set_name</em></dt>
<dd>
<p>Use to assign or invoke a backup set name. The backup set name must contain only printable characters and cannot have any spaces.</p>
</dd>
<dt class="hdlist1"><em>tables</em></dt>
<dd>
<p>List of tables (or a single table) to include in the backup set. Enter the table names as a comma-separated list. If no tables
are specified, all tables are included in the set.</p>
</dd>
</dl>
</div>
<div class="admonitionblock tip">
<table>
<tr>
<td class="icon">
<i class="fa icon-tip" title="Tip"></i>
</td>
<td class="content">
Maintain a log or other record of the case-sensitive backup set names and the corresponding tables in each set on a separate
or remote cluster, backup strategy. This information can help you in case of failure on the primary cluster.
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="br.set.usage"><a class="anchor" href="#br.set.usage"></a>87.4.3. Example of Usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup set add Q1Data TEAM3,TEAM_4</code></pre>
</div>
</div>
<div class="paragraph">
<p>Depending on the environment, this command results in <em>one</em> of the following actions:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>If the <code>Q1Data</code> backup set does not exist, a backup set containing tables <code>TEAM_3</code> and <code>TEAM_4</code> is created.</p>
</li>
<li>
<p>If the <code>Q1Data</code> backup set exists already, the tables <code>TEAM_3</code> and <code>TEAM_4</code> are added to the <code>Q1Data</code> backup set.</p>
</li>
</ul>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.administration"><a class="anchor" href="#br.administration"></a>88. Administration of Backup Images</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The <code>hbase backup</code> command has several subcommands that help with administering backup images as they accumulate. Most production
environments require recurring backups, so it is necessary to have utilities to help manage the data of the backup repository.
Some subcommands enable you to find information that can help identify backups that are relevant in a search for particular data.
You can also delete backup images.</p>
</div>
<div class="paragraph">
<p>The following list details each <code>hbase backup subcommand</code> that can help administer backups. Run the full command-subcommand line as
the HBase superuser.</p>
</div>
<div class="sect2">
<h3 id="br.managing.backup.progress"><a class="anchor" href="#br.managing.backup.progress"></a>88.1. Managing Backup Progress</h3>
<div class="paragraph">
<p>You can monitor a running backup in another terminal session by running the <em>hbase backup progress</em> command and specifying the backup ID as an argument.</p>
</div>
<div class="paragraph">
<p>For example, run the following command as hbase superuser to view the progress of a backup</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup progress &lt;backup_id&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.progress.positional.cli.arguments"><a class="anchor" href="#br.progress.positional.cli.arguments"></a>88.1.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_id</em></dt>
<dd>
<p>Specifies the backup that you want to monitor by seeing the progress information. The backupId is case-sensitive.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.progress.named.cli.arguments"><a class="anchor" href="#br.progress.named.cli.arguments"></a>88.1.2. Named Command-Line Arguments</h4>
<div class="paragraph">
<p>None.</p>
</div>
</div>
<div class="sect3">
<h4 id="br.progress.example"><a class="anchor" href="#br.progress.example"></a>88.1.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase backup progress backupId_1467823988425</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.managing.backup.history"><a class="anchor" href="#br.managing.backup.history"></a>88.2. Managing Backup History</h3>
<div class="paragraph">
<p>This command displays a log of backup sessions. The information for each session includes backup ID, type (full or incremental), the tables
in the backup, status, and start and end time. Specify the number of backup sessions to display with the optional -n argument.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup history &lt;backup_id&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.history.positional.cli.arguments"><a class="anchor" href="#br.history.positional.cli.arguments"></a>88.2.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_id</em></dt>
<dd>
<p>Specifies the backup that you want to monitor by seeing the progress information. The backupId is case-sensitive.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.history.named.cli.arguments"><a class="anchor" href="#br.history.named.cli.arguments"></a>88.2.2. Named Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>-n &lt;num_records&gt;</em></dt>
<dd>
<p>(Optional) The maximum number of backup records (Default: 10).</p>
</dd>
<dt class="hdlist1"><em>-p &lt;backup_root_path&gt;</em></dt>
<dd>
<p>The full filesystem URI of where backup images are stored.</p>
</dd>
<dt class="hdlist1"><em>-s &lt;backup_set_name&gt;</em></dt>
<dd>
<p>The name of the backup set to obtain history for. Mutually exclusive with the <em>-t</em> option.</p>
</dd>
<dt class="hdlist1"><em>-t</em> &lt;table_name&gt;</dt>
<dd>
<p>The name of table to obtain history for. Mutually exclusive with the <em>-s</em> option.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.history.backup.example"><a class="anchor" href="#br.history.backup.example"></a>88.2.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup history
<span class="error">$</span> hbase backup history -n <span class="integer">20</span>
<span class="error">$</span> hbase backup history -t WebIndexRecords</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.describe.backup"><a class="anchor" href="#br.describe.backup"></a>88.3. Describing a Backup Image</h3>
<div class="paragraph">
<p>This command can be used to obtain information about a specific backup image.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup describe &lt;backup_id&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.describe.backup.positional.cli.arguments"><a class="anchor" href="#br.describe.backup.positional.cli.arguments"></a>88.3.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_id</em></dt>
<dd>
<p>The ID of the backup image to describe.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.describe.backup.named.cli.arguments"><a class="anchor" href="#br.describe.backup.named.cli.arguments"></a>88.3.2. Named Command-Line Arguments</h4>
<div class="paragraph">
<p>None.</p>
</div>
</div>
<div class="sect3">
<h4 id="br.describe.backup.example"><a class="anchor" href="#br.describe.backup.example"></a>88.3.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup describe backupId_1467823988425</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.delete.backup"><a class="anchor" href="#br.delete.backup"></a>88.4. Deleting a Backup Image</h3>
<div class="paragraph">
<p>This command can be used to delete a backup image which is no longer needed.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup delete &lt;backup_id&gt;</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.delete.backup.positional.cli.arguments"><a class="anchor" href="#br.delete.backup.positional.cli.arguments"></a>88.4.1. Positional Command-Line Arguments</h4>
<div class="dlist">
<dl>
<dt class="hdlist1"><em>backup_id</em></dt>
<dd>
<p>The ID to the backup image which should be deleted.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="br.delete.backup.named.cli.arguments"><a class="anchor" href="#br.delete.backup.named.cli.arguments"></a>88.4.2. Named Command-Line Arguments</h4>
<div class="paragraph">
<p>None.</p>
</div>
</div>
<div class="sect3">
<h4 id="br.delete.backup.example"><a class="anchor" href="#br.delete.backup.example"></a>88.4.3. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup delete backupId_1467823988425</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.repair.backup"><a class="anchor" href="#br.repair.backup"></a>88.5. Backup Repair Command</h3>
<div class="paragraph">
<p>This command attempts to correct any inconsistencies in persisted backup metadata which exists as
the result of software errors or unhandled failure scenarios. While the backup implementation tries
to correct all errors on its own, this tool may be necessary in the cases where the system cannot
automatically recover on its own.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup repair</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="br.repair.backup.positional.cli.arguments"><a class="anchor" href="#br.repair.backup.positional.cli.arguments"></a>88.5.1. Positional Command-Line Arguments</h4>
<div class="paragraph">
<p>None.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="br.repair.backup.named.cli.arguments"><a class="anchor" href="#br.repair.backup.named.cli.arguments"></a>88.6. Named Command-Line Arguments</h3>
<div class="paragraph">
<p>None.</p>
</div>
<div class="sect3">
<h4 id="br.repair.backup.example"><a class="anchor" href="#br.repair.backup.example"></a>88.6.1. Example usage</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase backup repair</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.backup.configuration"><a class="anchor" href="#br.backup.configuration"></a>89. Configuration keys</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The backup and restore feature includes both required and optional configuration keys.</p>
</div>
<div class="sect2">
<h3 id="_required_properties"><a class="anchor" href="#_required_properties"></a>89.1. Required properties</h3>
<div class="paragraph">
<p><em>hbase.backup.enable</em>: Controls whether or not the feature is enabled (Default: <code>false</code>). Set this value to <code>true</code>.</p>
</div>
<div class="paragraph">
<p><em>hbase.master.logcleaner.plugins</em>: A comma-separated list of classes invoked when cleaning logs in the HBase Master. Set
this value to <code>org.apache.hadoop.hbase.backup.master.BackupLogCleaner</code> or append it to the current value.</p>
</div>
<div class="paragraph">
<p><em>hbase.procedure.master.classes</em>: A comma-separated list of classes invoked with the Procedure framework in the Master. Set
this value to <code>org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager</code> or append it to the current value.</p>
</div>
<div class="paragraph">
<p><em>hbase.procedure.regionserver.classes</em>: A comma-separated list of classes invoked with the Procedure framework in the RegionServer.
Set this value to <code>org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager</code> or append it to the current value.</p>
</div>
<div class="paragraph">
<p><em>hbase.coprocessor.region.classes</em>: A comma-separated list of RegionObservers deployed on tables. Set this value to
<code>org.apache.hadoop.hbase.backup.BackupObserver</code> or append it to the current value.</p>
</div>
<div class="paragraph">
<p><em>hbase.master.hfilecleaner.plugins</em>: A comma-separated list of HFileCleaners deployed on the Master. Set this value
to <code>org.apache.hadoop.hbase.backup.BackupHFileCleaner</code> or append it to the current value.</p>
</div>
</div>
<div class="sect2">
<h3 id="_optional_properties"><a class="anchor" href="#_optional_properties"></a>89.2. Optional properties</h3>
<div class="paragraph">
<p><em>hbase.backup.system.ttl</em>: The time-to-live in seconds of data in the <code>hbase:backup</code> tables (default: forever). This property
is only relevant prior to the creation of the <code>hbase:backup</code> table. Use the <code>alter</code> command in the HBase shell to modify the TTL
when this table already exists. See the <a href="#br.filesystem.growth.warning">below section</a> for more details on the impact of this
configuration property.</p>
</div>
<div class="paragraph">
<p><em>hbase.backup.attempts.max</em>: The number of attempts to perform when taking hbase table snapshots (default: 10).</p>
</div>
<div class="paragraph">
<p><em>hbase.backup.attempts.pause.ms</em>: The amount of time to wait between failed snapshot attempts in milliseconds (default: 10000).</p>
</div>
<div class="paragraph">
<p><em>hbase.backup.logroll.timeout.millis</em>: The amount of time (in milliseconds) to wait for RegionServers to execute a WAL rolling
in the Master&#8217;s procedure framework (default: 30000).</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.best.practices"><a class="anchor" href="#br.best.practices"></a>90. Best Practices</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_formulate_a_restore_strategy_and_test_it"><a class="anchor" href="#_formulate_a_restore_strategy_and_test_it"></a>90.1. Formulate a restore strategy and test it.</h3>
<div class="paragraph">
<p>Before you rely on a backup and restore strategy for your production environment, identify how backups must be performed,
and more importantly, how restores must be performed. Test the plan to ensure that it is workable.
At a minimum, store backup data from a production cluster on a different cluster or server. To further safeguard the data,
use a backup location that is at a different physical location.</p>
</div>
<div class="paragraph">
<p>If you have a unrecoverable loss of data on your primary production cluster as a result of computer system issues, you may
be able to restore the data from a different cluster or server at the same site. However, a disaster that destroys the whole
site renders locally stored backups useless. Consider storing the backup data and necessary resources (both computing capacity
and operator expertise) to restore the data at a site sufficiently remote from the production site. In the case of a catastrophe
at the whole primary site (fire, earthquake, etc.), the remote backup site can be very valuable.</p>
</div>
</div>
<div class="sect2">
<h3 id="_secure_a_full_backup_image_first"><a class="anchor" href="#_secure_a_full_backup_image_first"></a>90.2. Secure a full backup image first.</h3>
<div class="paragraph">
<p>As a baseline, you must complete a full backup of HBase data at least once before you can rely on incremental backups. The full
backup should be stored outside of the source cluster. To ensure complete dataset recovery, you must run the restore utility
with the option to restore baseline full backup. The full backup is the foundation of your dataset. Incremental backup data
is applied on top of the full backup during the restore operation to return you to the point in time when backup was last taken.</p>
</div>
</div>
<div class="sect2">
<h3 id="_define_and_use_backup_sets_for_groups_of_tables_that_are_logical_subsets_of_the_entire_dataset"><a class="anchor" href="#_define_and_use_backup_sets_for_groups_of_tables_that_are_logical_subsets_of_the_entire_dataset"></a>90.3. Define and use backup sets for groups of tables that are logical subsets of the entire dataset.</h3>
<div class="paragraph">
<p>You can group tables into an object called a backup set. A backup set can save time when you have a particular group of tables
that you expect to repeatedly back up or restore.</p>
</div>
<div class="paragraph">
<p>When you create a backup set, you type table names to include in the group. The backup set includes not only groups of related
tables, but also retains the HBase backup metadata. Afterwards, you can invoke the backup set name to indicate what tables apply
to the command execution instead of entering all the table names individually.</p>
</div>
</div>
<div class="sect2">
<h3 id="_document_the_backup_and_restore_strategy_and_ideally_log_information_about_each_backup"><a class="anchor" href="#_document_the_backup_and_restore_strategy_and_ideally_log_information_about_each_backup"></a>90.4. Document the backup and restore strategy, and ideally log information about each backup.</h3>
<div class="paragraph">
<p>Document the whole process so that the knowledge base can transfer to new administrators after employee turnover. As an extra
safety precaution, also log the calendar date, time, and other relevant details about the data of each backup. This metadata
can potentially help locate a particular dataset in case of source cluster failure or primary site disaster. Maintain duplicate
copies of all documentation: one copy at the production cluster site and another at the backup location or wherever it can be
accessed by an administrator remotely from the production cluster.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.s3.backup.scenario"><a class="anchor" href="#br.s3.backup.scenario"></a>91. Scenario: Safeguarding Application Datasets on Amazon S3</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This scenario describes how a hypothetical retail business uses backups to safeguard application data and then restore the dataset
after failure.</p>
</div>
<div class="paragraph">
<p>The HBase administration team uses backup sets to store data from a group of tables that have interrelated information for an
application called green. In this example, one table contains transaction records and the other contains customer details. The
two tables need to be backed up and be recoverable as a group.</p>
</div>
<div class="paragraph">
<p>The admin team also wants to ensure daily backups occur automatically.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/backup-app-components.png" alt="backup app components">
</div>
<div class="title">Figure 8. Tables Composing The Backup Set</div>
</div>
<div class="paragraph">
<p>The following is an outline of the steps and examples of commands that are used to backup the data for the <em>green</em> application and
to recover the data later. All commands are run when logged in as HBase superuser.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A backup set called <em>green_set</em> is created as an alias for both the transactions table and the customer table. The backup set can
be used for all operations to avoid typing each table name. The backup set name is case-sensitive and should be formed with only
printable characters and without spaces.</p>
<div class="literalblock">
<div class="content">
<pre>$ hbase backup set add green_set transactions
$ hbase backup set add green_set customer</pre>
</div>
</div>
</li>
<li>
<p>The first backup of green_set data must be a full backup. The following command example shows how credentials are passed to Amazon
S3 and specifies the file system with the s3a: prefix.</p>
<div class="literalblock">
<div class="content">
<pre>$ ACCESS_KEY=ABCDEFGHIJKLMNOPQRST
$ SECRET_KEY=123456789abcdefghijklmnopqrstuvwxyzABCD
$ sudo -u hbase hbase backup create full\
s3a://$ACCESS_KEY:SECRET_KEY@prodhbasebackups/backups -s green_set</pre>
</div>
</div>
</li>
<li>
<p>Incremental backups should be run according to a schedule that ensures essential data recovery in the event of a catastrophe. At
this retail company, the HBase admin team decides that automated daily backups secures the data sufficiently. The team decides that
they can implement this by modifying an existing Cron job that is defined in <code>/etc/crontab</code>. Consequently, IT modifies the Cron job
by adding the following line:</p>
<div class="literalblock">
<div class="content">
<pre>@daily hbase hbase backup create incremental s3a://$ACCESS_KEY:$SECRET_KEY@prodhbasebackups/backups -s green_set</pre>
</div>
</div>
</li>
<li>
<p>A catastrophic IT incident disables the production cluster that the green application uses. An HBase system administrator of the
backup cluster must restore the <em>green_set</em> dataset to the point in time closest to the recovery objective.</p>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If the administrator of the backup HBase cluster has the backup ID with relevant details in accessible records, the following
search with the <code>hdfs dfs -ls</code> command and manually scanning the backup ID list can be bypassed. Consider continuously maintaining
and protecting a detailed log of backup IDs outside the production cluster in your environment.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The HBase administrator runs the following command on the directory where backups are stored to print the list of successful backup
IDs on the console:</p>
</div>
<div class="literalblock">
<div class="content">
<pre>`hdfs dfs -ls -t /prodhbasebackups/backups`</pre>
</div>
</div>
</li>
<li>
<p>The admin scans the list to see which backup was created at a date and time closest to the recovery objective. To do this, the
admin converts the calendar timestamp of the recovery point in time to Unix time because backup IDs are uniquely identified with
Unix time. The backup IDs are listed in reverse chronological order, meaning the most recent successful backup appears first.</p>
<div class="paragraph">
<p>The admin notices that the following line in the command output corresponds with the <em>green_set</em> backup that needs to be restored:</p>
</div>
<div class="literalblock">
<div class="content">
<pre>/prodhbasebackups/backups/backup_1467823988425`</pre>
</div>
</div>
</li>
<li>
<p>The admin restores green_set invoking the backup ID and the -overwrite option. The -overwrite option truncates all existing data
in the destination and populates the tables with data from the backup dataset. Without this flag, the backup data is appended to the
existing data in the destination. In this case, the admin decides to overwrite the data because it is corrupted.</p>
<div class="literalblock">
<div class="content">
<pre>$ sudo -u hbase hbase restore -s green_set \
s3a://$ACCESS_KEY:$SECRET_KEY@prodhbasebackups/backups backup_1467823988425 \ -overwrite</pre>
</div>
</div>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.data.security"><a class="anchor" href="#br.data.security"></a>92. Security of Backup Data</h2>
<div class="sectionbody">
<div class="paragraph">
<p>With this feature which makes copying data to remote locations, it&#8217;s important to take a moment to clearly state the procedural
concerns that exist around data security. Like the HBase replication feature, backup and restore provides the constructs to automatically
copy data from within a corporate boundary to some system outside of that boundary. It is imperative when storing sensitive data that with backup and restore, much
less any feature which extracts data from HBase, the locations to which data is being sent has undergone a security audit to ensure
that only authenticated users are allowed to access that data.</p>
</div>
<div class="paragraph">
<p>For example, with the above example of backing up data to S3, it is of the utmost importance that the proper permissions are assigned
to the S3 bucket to ensure that only a minimum set of authorized users are allowed to access this data. Because the data is no longer
being accessed via HBase, and its authentication and authorization controls, we must ensure that the filesystem storing that data is
providing a comparable level of security. This is a manual step which users <strong>must</strong> implement on their own.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.technical.details"><a class="anchor" href="#br.technical.details"></a>93. Technical Details of Incremental Backup and Restore</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase incremental backups enable more efficient capture of HBase table images than previous attempts at serial backup and restore
solutions, such as those that only used HBase Export and Import APIs. Incremental backups use Write Ahead Logs (WALs) to capture
the data changes since the previous backup was created. A WAL roll (create new WALs) is executed across all RegionServers to track
the WALs that need to be in the backup.</p>
</div>
<div class="paragraph">
<p>After the incremental backup image is created, the source backup files usually are on same node as the data source. A process similar
to the DistCp (distributed copy) tool is used to move the source backup files to the target file systems. When a table restore operation
starts, a two-step process is initiated. First, the full backup is restored from the full backup image. Second, all WAL files from
incremental backups between the last full backup and the incremental backup being restored are converted to HFiles, which the HBase
Bulk Load utility automatically imports as restored data in the table.</p>
</div>
<div class="paragraph">
<p>You can only restore on a live HBase cluster because the data must be redistributed to complete the restore operation successfully.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.filesystem.growth.warning"><a class="anchor" href="#br.filesystem.growth.warning"></a>94. A Warning on File System Growth</h2>
<div class="sectionbody">
<div class="paragraph">
<p>As a reminder, incremental backups are implemented via retaining the write-ahead logs which HBase primarily uses for data durability.
Thus, to ensure that all data needing to be included in a backup is still available in the system, the HBase backup and restore feature
retains all write-ahead logs since the last backup until the next incremental backup is executed.</p>
</div>
<div class="paragraph">
<p>Like HBase Snapshots, this can have an expectedly large impact on the HDFS usage of HBase for high volume tables. Take care in enabling
and using the backup and restore feature, specifically with a mind to removing backup sessions when they are not actively being used.</p>
</div>
<div class="paragraph">
<p>The only automated, upper-bound on retained write-ahead logs for backup and restore is based on the TTL of the <code>hbase:backup</code> system table which,
as of the time this document is written, is infinite (backup table entries are never automatically deleted). This requires that administrators
perform backups on a schedule whose frequency is relative to the amount of available space on HDFS (e.g. less available HDFS space requires
more aggressive backup merges and deletions). As a reminder, the TTL can be altered on the <code>hbase:backup</code> table using the <code>alter</code> command
in the HBase shell. Modifying the configuration property <code>hbase.backup.system.ttl</code> in hbase-site.xml after the system table exists has no effect.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.backup.capacity.planning"><a class="anchor" href="#br.backup.capacity.planning"></a>95. Capacity Planning</h2>
<div class="sectionbody">
<div class="paragraph">
<p>When designing a distributed system deployment, it is critical that some basic mathmatical rigor is executed to ensure sufficient computational
capacity is available given the data and software requirements of the system. For this feature, the availability of network capacity is the largest
bottleneck when estimating the performance of some implementation of backup and restore. The second most costly function is the speed at which
data can be read/written.</p>
</div>
<div class="sect2">
<h3 id="_full_backups"><a class="anchor" href="#_full_backups"></a>95.1. Full Backups</h3>
<div class="paragraph">
<p>To estimate the duration of a full backup, we have to understand the general actions which are invoked:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Write-ahead log roll on each RegionServer: ones to tens of seconds per RegionServer in parallel. Relative to the load on each RegionServer.</p>
</li>
<li>
<p>Take an HBase snapshot of the table(s): tens of seconds. Relative to the number of regions and files that comprise the table.</p>
</li>
<li>
<p>Export the snapshot to the destination: see below. Relative to the size of the data and the network bandwidth to the destination.</p>
</li>
</ul>
</div>
<div id="br.export.snapshot.cost" class="paragraph">
<p>To approximate how long the final step will take, we have to make some assumptions on hardware. Be aware that these will <strong>not</strong> be accurate for your
system&#8201;&#8212;&#8201;these are numbers that your or your administrator know for your system. Let&#8217;s say the speed of reading data from HDFS on a single node is
capped at 80MB/s (across all Mappers that run on that host), a modern network interface controller (NIC) supports 10Gb/s, the top-of-rack switch can
handle 40Gb/s, and the WAN between your clusters is 10Gb/s. This means that you can only ship data to your remote at a speed of 1.25GB/s&#8201;&#8212;&#8201;meaning
that 16 nodes (<code>1.25 * 1024 / 80 = 16</code>) participating in the ExportSnapshot should be able to fully saturate the link between clusters. With more
nodes in the cluster, we can still saturate the network but at a lesser impact on any one node which helps ensure local SLAs are made. If the size
of the snapshot is 10TB, this would full backup would take in the ballpark of 2.5 hours (<code>10 * 1024 / 1.25 / (60 * 60) = 2.23hrs</code>)</p>
</div>
<div class="paragraph">
<p>As a general statement, it is very likely that the WAN bandwidth between your local cluster and the remote storage is the largest
bottleneck to the speed of a full backup.</p>
</div>
<div class="paragraph">
<p>When the concern is restricting the computational impact of backups to a "production system", the above formulas can be reused with the optional
command-line arguments to <code>hbase backup create</code>: <code>-b</code>, <code>-w</code>, <code>-q</code>. The <code>-b</code> option defines the bandwidth at which each worker (Mapper) would
write data. The <code>-w</code> argument limits the number of workers that would be spawned in the DistCp job. The <code>-q</code> allows the user to specify a YARN
queue which can limit the specific nodes where the workers will be spawned&#8201;&#8212;&#8201;this can quarantine the backup workers performing the copy to
a set of non-critical nodes. Relating the <code>-b</code> and <code>-w</code> options to our earlier equations: <code>-b</code> would be used to restrict each node from reading
data at the full 80MB/s and <code>-w</code> is used to limit the job from spawning 16 worker tasks.</p>
</div>
</div>
<div class="sect2">
<h3 id="_incremental_backup"><a class="anchor" href="#_incremental_backup"></a>95.2. Incremental Backup</h3>
<div class="paragraph">
<p>Like we did for full backups, we have to understand the incremental backup process to approximate its runtime and cost.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Identify new write-ahead logs since last full or incremental backup: negligible. Apriori knowledge from the backup system table(s).</p>
</li>
<li>
<p>Read, filter, and write "minimized" HFiles equivalent to the WALs: dominated by the speed of writing data. Relative to write speed of HDFS.</p>
</li>
<li>
<p>DistCp the HFiles to the destination: <a href="#br.export.snapshot.cost">see above</a>.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>For the second step, the dominating cost of this operation would be the re-writing the data (under the assumption that a majority of the
data in the WAL is preserved). In this case, we can assume an aggregate write speed of 30MB/s per node. Continuing our 16-node cluster example,
this would require approximately 15 minutes to perform this step for 50GB of data (50 * 1024 / 60 / 60 = 14.2). The amount of time to start the
DistCp MapReduce job would likely dominate the actual time taken to copy the data (50 / 1.25 = 40 seconds) and can be ignored.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="br.limitations"><a class="anchor" href="#br.limitations"></a>96. Limitations of the Backup and Restore Utility</h2>
<div class="sectionbody">
<div class="paragraph">
<p><strong>Serial backup operations</strong></p>
</div>
<div class="paragraph">
<p>Backup operations cannot be run concurrently. An operation includes actions like create, delete, restore, and merge. Only one active backup session is supported. <a href="https://issues.apache.org/jira/browse/HBASE-16391">HBASE-16391</a>
will introduce multiple-backup sessions support.</p>
</div>
<div class="paragraph">
<p><strong>No means to cancel backups</strong></p>
</div>
<div class="paragraph">
<p>Both backup and restore operations cannot be canceled. (<a href="https://issues.apache.org/jira/browse/HBASE-15997">HBASE-15997</a>, <a href="https://issues.apache.org/jira/browse/HBASE-15998">HBASE-15998</a>).
The workaround to cancel a backup would be to kill the client-side backup command (<code>control-C</code>), ensure all relevant MapReduce jobs have exited, and then
run the <code>hbase backup repair</code> command to ensure the system backup metadata is consistent.</p>
</div>
<div class="paragraph">
<p><strong>Backups can only be saved to a single location</strong></p>
</div>
<div class="paragraph">
<p>Copying backup information to multiple locations is an exercise left to the user. <a href="https://issues.apache.org/jira/browse/HBASE-15476">HBASE-15476</a> will
introduce the ability to specify multiple-backup destinations intrinsically.</p>
</div>
<div class="paragraph">
<p><strong>HBase superuser access is required</strong></p>
</div>
<div class="paragraph">
<p>Only an HBase superuser (e.g. hbase) is allowed to perform backup/restore, can pose a problem for shared HBase installations. Current mitigations would require
coordination with system administrators to build and deploy a backup and restore strategy (<a href="https://issues.apache.org/jira/browse/HBASE-14138">HBASE-14138</a>).</p>
</div>
<div class="paragraph">
<p><strong>Backup restoration is an online operation</strong></p>
</div>
<div class="paragraph">
<p>To perform a restore from a backup, it requires that the HBase cluster is online as a caveat of the current implementation (<a href="https://issues.apache.org/jira/browse/HBASE-16573">HBASE-16573</a>).</p>
</div>
<div class="paragraph">
<p><strong>Some operations may fail and require re-run</strong></p>
</div>
<div class="paragraph">
<p>The HBase backup feature is primarily client driven. While there is the standard HBase retry logic built into the HBase Connection, persistent errors in executing operations
may propagate back to the client (e.g. snapshot failure due to region splits). The backup implementation should be moved from client-side into the ProcedureV2 framework
in the future which would provide additional robustness around transient/retryable failures. The <code>hbase backup repair</code> command is meant to correct states which the system
cannot automatically detect and recover from.</p>
</div>
<div class="paragraph">
<p><strong>Avoidance of declaration of public API</strong></p>
</div>
<div class="paragraph">
<p>While the Java API to interact with this feature exists and its implementation is separated from an interface, insufficient rigor has been applied to determine if
it is exactly what we intend to ship to users. As such, it is marked as for a <code>Private</code> audience with the expectation that, as users begin to try the feature, there
will be modifications that would necessitate breaking compatibility (<a href="https://issues.apache.org/jira/browse/HBASE-17517">HBASE-17517</a>).</p>
</div>
<div class="paragraph">
<p><strong>Lack of global metrics for backup and restore</strong></p>
</div>
<div class="paragraph">
<p>Individual backup and restore operations contain metrics about the amount of work the operation included, but there is no centralized location (e.g. the Master UI)
which present information for consumption (<a href="https://issues.apache.org/jira/browse/HBASE-16565">HBASE-16565</a>).</p>
</div>
</div>
</div>
<h1 id="syncreplication" class="sect0"><a class="anchor" href="#syncreplication"></a>Synchronous Replication</h1>
<div class="sect1">
<h2 id="_background"><a class="anchor" href="#_background"></a>97. Background</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The current <a href="#_cluster_replication">replication</a> in HBase in asynchronous. So if the master cluster crashes, the slave cluster may not have the
newest data. If users want strong consistency then they can not switch to the slave cluster.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_design"><a class="anchor" href="#_design"></a>98. Design</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Please see the design doc on <a href="https://issues.apache.org/jira/browse/HBASE-19064">HBASE-19064</a></p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_operation_and_maintenance"><a class="anchor" href="#_operation_and_maintenance"></a>99. Operation and maintenance</h2>
<div class="sectionbody">
<div class="dlist">
<dl>
<dt class="hdlist1">Case.1 Setup two synchronous replication clusters</dt>
<dd>
<div class="ulist">
<ul>
<li>
<p>Add a synchronous peer in both source cluster and peer cluster.</p>
</li>
</ul>
</div>
</dd>
</dl>
</div>
<div class="paragraph">
<p>For source cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; add_peer <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="constant">CLUSTER_KEY</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">lg-hadoop-tst-st01.bj:10010,lg-hadoop-tst-st02.bj:10010,lg-hadoop-tst-st03.bj:10010:/hbase/test-hbase-slave</span><span class="delimiter">'</span></span>, <span class="constant">REMOTE_WAL_DIR</span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">hdfs://lg-hadoop-tst-st01.bj:20100/hbase/test-hbase-slave/remoteWALs</span><span class="delimiter">'</span></span>, <span class="constant">TABLE_CFS</span> =&gt; {<span class="string"><span class="delimiter">&quot;</span><span class="content">ycsb-test</span><span class="delimiter">&quot;</span></span>=&gt;[]}</code></pre>
</div>
</div>
<div class="paragraph">
<p>For peer cluster:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; add_peer <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="constant">CLUSTER_KEY</span> =&gt; <span class="string"><span class="delimiter">'</span><span class="content">lg-hadoop-tst-st01.bj:10010,lg-hadoop-tst-st02.bj:10010,lg-hadoop-tst-st03.bj:10010:/hbase/test-hbase</span><span class="delimiter">'</span></span>, <span class="constant">REMOTE_WAL_DIR</span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">hdfs://lg-hadoop-tst-st01.bj:20100/hbase/test-hbase/remoteWALs</span><span class="delimiter">'</span></span>, <span class="constant">TABLE_CFS</span> =&gt; {<span class="string"><span class="delimiter">&quot;</span><span class="content">ycsb-test</span><span class="delimiter">&quot;</span></span>=&gt;[]}</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
For synchronous replication, the current implementation require that we have the same peer id for both source
and peer cluster. Another thing that need attention is: the peer does not support cluster-level, namespace-level, or
cf-level replication, only support table-level replication now.
</td>
</tr>
</table>
</div>
<div class="ulist">
<ul>
<li>
<p>Transit the peer cluster to be STANDBY state</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">STANDBY</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Transit the source cluster to be ACTIVE state</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">ACTIVE</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Now, the synchronous replication has been set up successfully. the HBase client can only request to source cluster, if
request to peer cluster, the peer cluster which is STANDBY state now will reject the read/write requests.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Case.2 How to operate when standby cluster crashed</dt>
<dd>
<p>If the standby cluster has been crashed, it will fail to write remote WAL for the active cluster. So we need to transit
the source cluster to DOWNGRANDE_ACTIVE state, which means source cluster won&#8217;t write any remote WAL any more, but
the normal replication (asynchronous Replication) can still work fine, it queue the newly written WALs, but the
replication block until the peer cluster come back.</p>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">DOWNGRADE_ACTIVE</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Once the peer cluster come back, we can just transit the source cluster to ACTIVE, to ensure that the replication will be
synchronous.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">ACTIVE</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Case.3 How to operate when active cluster crashed</dt>
<dd>
<p>If the active cluster has been crashed (it may be not reachable now), so let&#8217;s just transit the standby cluster to
DOWNGRANDE_ACTIVE state, and after that, we should redirect all the requests from client to the DOWNGRADE_ACTIVE cluster.</p>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">DOWNGRADE_ACTIVE</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>If the crashed cluster come back again, we just need to transit it to STANDBY directly. Otherwise if you transit the
cluster to DOWNGRADE_ACTIVE, the original ACTIVE cluster may have redundant data compared to the current ACTIVE
cluster. Because we designed to write source cluster WALs and remote cluster WALs concurrently, so it&#8217;s possible that
the source cluster WALs has more data than the remote cluster, which result in data inconsistency. The procedure of
transiting ACTIVE to STANDBY has no problem, because we&#8217;ll skip to replay the original WALs.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">STANDBY</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>After that, we can promote the DOWNGRADE_ACTIVE cluster to ACTIVE now, to ensure that the replication will be synchronous.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; transit_peer_sync_replication_state <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">ACTIVE</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</div>
</div>
<h1 id="hbase_apis" class="sect0"><a class="anchor" href="#hbase_apis"></a>Apache HBase APIs</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>This chapter provides information about performing operations using HBase native APIs.
This information is not exhaustive, and provides a quick reference in addition to the <a href="https://hbase.apache.org/apidocs/index.html">User API Reference</a>.
The examples here are not comprehensive or complete, and should be used for purposes of illustration only.</p>
</div>
<div class="paragraph">
<p>Apache HBase also works with multiple external APIs.
See <a href="#external_apis">Apache HBase External APIs</a> for more information.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_examples"><a class="anchor" href="#_examples"></a>100. Examples</h2>
<div class="sectionbody">
<div class="exampleblock">
<div class="title">Example 28. Create, modify and delete a Table Using Java</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">package</span> <span class="namespace">com.example.hbase.admin</span>;
<span class="keyword">import</span> <span class="include">java.io.IOException</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.conf.Configuration</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.fs.Path</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.HBaseConfiguration</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.HColumnDescriptor</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.HConstants</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.HTableDescriptor</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.TableName</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.client.Admin</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.client.Connection</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.client.ConnectionFactory</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.io.compress.Compression.Algorithm</span>;
<span class="directive">public</span> <span class="type">class</span> <span class="class">Example</span> {
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="predefined-type">String</span> TABLE_NAME = <span class="string"><span class="delimiter">&quot;</span><span class="content">MY_TABLE_NAME_TOO</span><span class="delimiter">&quot;</span></span>;
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="predefined-type">String</span> CF_DEFAULT = <span class="string"><span class="delimiter">&quot;</span><span class="content">DEFAULT_COLUMN_FAMILY</span><span class="delimiter">&quot;</span></span>;
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> createOrOverwrite(Admin admin, HTableDescriptor table) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">if</span> (admin.tableExists(table.getTableName())) {
admin.disableTable(table.getTableName());
admin.deleteTable(table.getTableName());
}
admin.createTable(table);
}
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> createSchemaTables(<span class="predefined-type">Configuration</span> config) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(config);
Admin admin = connection.getAdmin()) {
HTableDescriptor table = <span class="keyword">new</span> HTableDescriptor(TableName.valueOf(TABLE_NAME));
table.addFamily(<span class="keyword">new</span> HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.NONE));
<span class="predefined-type">System</span>.out.print(<span class="string"><span class="delimiter">&quot;</span><span class="content">Creating table. </span><span class="delimiter">&quot;</span></span>);
createOrOverwrite(admin, table);
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content"> Done.</span><span class="delimiter">&quot;</span></span>);
}
}
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> modifySchema (<span class="predefined-type">Configuration</span> config) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">try</span> (<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(config);
Admin admin = connection.getAdmin()) {
TableName tableName = TableName.valueOf(TABLE_NAME);
<span class="keyword">if</span> (!admin.tableExists(tableName)) {
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">Table does not exist.</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">System</span>.exit(-<span class="integer">1</span>);
}
HTableDescriptor table = admin.getTableDescriptor(tableName);
<span class="comment">// Update existing table</span>
HColumnDescriptor newColumn = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">NEWCF</span><span class="delimiter">&quot;</span></span>);
newColumn.setCompactionCompressionType(Algorithm.GZ);
newColumn.setMaxVersions(HConstants.ALL_VERSIONS);
admin.addColumn(tableName, newColumn);
<span class="comment">// Update existing column family</span>
HColumnDescriptor existingColumn = <span class="keyword">new</span> HColumnDescriptor(CF_DEFAULT);
existingColumn.setCompactionCompressionType(Algorithm.GZ);
existingColumn.setMaxVersions(HConstants.ALL_VERSIONS);
table.modifyFamily(existingColumn);
admin.modifyTable(tableName, table);
<span class="comment">// Disable an existing table</span>
admin.disableTable(tableName);
<span class="comment">// Delete an existing column family</span>
admin.deleteColumn(tableName, CF_DEFAULT.getBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">UTF-8</span><span class="delimiter">&quot;</span></span>));
<span class="comment">// Delete a table (Need to be disabled first)</span>
admin.deleteTable(tableName);
}
}
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> main(<span class="predefined-type">String</span>... args) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
<span class="comment">//Add any necessary configuration files (hbase-site.xml, core-site.xml)</span>
config.addResource(<span class="keyword">new</span> Path(<span class="predefined-type">System</span>.getenv(<span class="string"><span class="delimiter">&quot;</span><span class="content">HBASE_CONF_DIR</span><span class="delimiter">&quot;</span></span>), <span class="string"><span class="delimiter">&quot;</span><span class="content">hbase-site.xml</span><span class="delimiter">&quot;</span></span>));
config.addResource(<span class="keyword">new</span> Path(<span class="predefined-type">System</span>.getenv(<span class="string"><span class="delimiter">&quot;</span><span class="content">HADOOP_CONF_DIR</span><span class="delimiter">&quot;</span></span>), <span class="string"><span class="delimiter">&quot;</span><span class="content">core-site.xml</span><span class="delimiter">&quot;</span></span>));
createSchemaTables(config);
modifySchema(config);
}
}</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<h1 id="external_apis" class="sect0"><a class="anchor" href="#external_apis"></a>Apache HBase External APIs</h1>
<div class="openblock partintro">
<div class="content">
This chapter will cover access to Apache HBase either through non-Java languages and
through custom protocols. For information on using the native HBase APIs, refer to
<a href="https://hbase.apache.org/apidocs/index.html">User API Reference</a> and the
<a href="#hbase_apis">HBase APIs</a> chapter.
</div>
</div>
<div class="sect1">
<h2 id="_rest"><a class="anchor" href="#_rest"></a>101. REST</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Representational State Transfer (REST) was introduced in 2000 in the doctoral
dissertation of Roy Fielding, one of the principal authors of the HTTP specification.</p>
</div>
<div class="paragraph">
<p>REST itself is out of the scope of this documentation, but in general, REST allows
client-server interactions via an API that is tied to the URL itself. This section
discusses how to configure and run the REST server included with HBase, which exposes
HBase tables, rows, cells, and metadata as URL specified resources.
There is also a nice series of blogs on
<a href="http://blog.cloudera.com/blog/2013/03/how-to-use-the-apache-hbase-rest-interface-part-1/">How-to: Use the Apache HBase REST Interface</a>
by Jesse Anderson.</p>
</div>
<div class="sect2">
<h3 id="_starting_and_stopping_the_rest_server"><a class="anchor" href="#_starting_and_stopping_the_rest_server"></a>101.1. Starting and Stopping the REST Server</h3>
<div class="paragraph">
<p>The included REST server can run as a daemon which starts an embedded Jetty
servlet container and deploys the servlet into it. Use one of the following commands
to start the REST server in the foreground or background. The port is optional, and
defaults to 8080.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash"># Foreground
$ bin/hbase rest start -p &lt;port&gt;
# Background, logging to a file in $HBASE_LOGS_DIR
$ bin/hbase-daemon.sh start rest -p &lt;port&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>To stop the REST server, use Ctrl-C if you were running it in the foreground, or the
following command if you were running it in the background.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase-daemon.sh stop rest</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_configuring_the_rest_server_and_client"><a class="anchor" href="#_configuring_the_rest_server_and_client"></a>101.2. Configuring the REST Server and Client</h3>
<div class="paragraph">
<p>For information about configuring the REST server and client for SSL, as well as <code>doAs</code>
impersonation for the REST server, see <a href="#security.gateway.thrift">Configure the Thrift Gateway to Authenticate on Behalf of the Client</a> and other portions
of the <a href="#security">Securing Apache HBase</a> chapter.</p>
</div>
</div>
<div class="sect2">
<h3 id="_using_rest_endpoints"><a class="anchor" href="#_using_rest_endpoints"></a>101.3. Using REST Endpoints</h3>
<div class="paragraph">
<p>The following examples use the placeholder server http://example.com:8000, and
the following commands can all be run using <code>curl</code> or <code>wget</code> commands. You can request
plain text (the default), XML , or JSON output by adding no header for plain text,
or the header "Accept: text/xml" for XML, "Accept: application/json" for JSON, or
"Accept: application/x-protobuf" to for protocol buffers.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Unless specified, use <code>GET</code> requests for queries, <code>PUT</code> or <code>POST</code> requests for
creation or mutation, and <code>DELETE</code> for deletion.
</td>
</tr>
</table>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 13. Cluster-Wide Endpoints</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/version/cluster</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version of HBase running on this cluster</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/version/cluster"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/status/cluster</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Cluster status</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/status/cluster"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">List of all non-system tables</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/"</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 14. Namespace Endpoints</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">List all namespaces</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/namespaces/"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces/<em>namespace</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Describe a specific namespace</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/namespaces/special_ns"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces/<em>namespace</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>POST</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Create a new namespace</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X POST \
-H "Accept: text/xml" \
"example.com:8000/namespaces/special_ns"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces/<em>namespace</em>/tables</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">List all tables in a specific namespace</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/namespaces/special_ns/tables"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces/<em>namespace</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Alter an existing namespace. Currently not used.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
"http://example.com:8000/namespaces/special_ns</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/namespaces/<em>namespace</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Delete a namespace. The namespace must be empty.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
"example.com:8000/namespaces/special_ns"</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 15. Table Endpoints</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/schema</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Describe the schema of the specified table.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/schema"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/schema</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>POST</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Update an existing table with the provided schema fragment</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X POST \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8"?&gt;&lt;TableSchema name="users"&gt;&lt;ColumnSchema name="cf" KEEP_DELETED_CELLS="true" /&gt;&lt;/TableSchema&gt;' \
"http://example.com:8000/users/schema"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/schema</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Create a new table, or replace an existing table&#8217;s schema</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8"?&gt;&lt;TableSchema name="users"&gt;&lt;ColumnSchema name="cf" /&gt;&lt;/TableSchema&gt;' \
"http://example.com:8000/users/schema"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/schema</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Delete the table. You must use the <code>/<em>table</em>/schema</code> endpoint, not just <code>/<em>table</em>/</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
"http://example.com:8000/users/schema"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/regions</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">List the table regions</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/regions</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 16. Endpoints for <code>Get</code> Operations</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Get all columns of a single row. Values are Base-64 encoded. This requires the "Accept" request header with a type that can hold multiple columns (like xml, json or protobuf).</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/row1"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row</em>/<em>column:qualifier</em>/<em>timestamp</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Get the value of a single column. Values are Base-64 encoded.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/row1/cf:a/1458586888395"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row</em>/<em>column:qualifier</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Get the value of a single column. Values are Base-64 encoded.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/row1/cf:a"
curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/row1/cf:a/"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row</em>/<em>column:qualifier</em>/?v=<em>number_of_versions</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Multi-Get a specified number of versions of a given cell. Values are Base-64 encoded.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/row1/cf:a?v=2"</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 17. Endpoints for <code>Scan</code> Operations</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/scanner/</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Get a Scanner object. Required by all other Scan operations. Adjust the batch parameter
to the number of rows the scan should return in a batch. See the next example for
adding filters to your scanner. The scanner endpoint URL is returned as the <code>Location</code>
in the HTTP response. The other examples in this table assume that the scanner endpoint
is <code>http://example.com:8000/users/scanner/145869072824375522207</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;Scanner batch="1"/&gt;' \
"http://example.com:8000/users/scanner/"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/scanner/</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">To supply filters to the Scanner object or configure the
Scanner in any other way, you can create a text file and add
your filter to the file. For example, to return only rows for
which keys start with &lt;codeph&gt;u123&lt;/codeph&gt; and use a batch size
of 100, the filter file would look like this:
</p><p class="tableblock">[source,xml]
----
&lt;Scanner batch="100"&gt;
&lt;filter&gt;
{
"type": "PrefixFilter",
"value": "u123"
}
&lt;/filter&gt;
&lt;/Scanner&gt;
----
</p><p class="tableblock">Pass the file to the <code>-d</code> argument of the <code>curl</code> request.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
-H "Content-Type:text/xml" \
-d @filter.txt \
"http://example.com:8000/users/scanner/"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/scanner/<em>scanner-id</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>GET</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Get the next batch from the scanner. Cell values are byte-encoded. If the scanner
has been exhausted, HTTP status <code>204</code> is returned.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X GET \
-H "Accept: text/xml" \
"http://example.com:8000/users/scanner/145869072824375522207"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code><em>table</em>/scanner/<em>scanner-id</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Deletes the scanner and frees the resources it used.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
"http://example.com:8000/users/scanner/145869072824375522207"</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 18. Endpoints for <code>Put</code> Operations</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em></code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Write a row to a table. The row, column qualifier, and value must each be Base-64
encoded. To encode a string, use the <code>base64</code> command-line utility. To decode the
string, use <code>base64 -d</code>. The payload is in the <code>--data</code> argument, and the <code>/users/fakerow</code>
value is a placeholder. Insert multiple rows by adding them to the <code>&lt;CellSet&gt;</code>
element. You can also save the data to be inserted to a file and pass it to the <code>-d</code>
parameter with syntax like <code>-d @filename.txt</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93NQo="&gt;&lt;Cell column="Y2Y6ZQo="&gt;dmFsdWU1Cg==&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/fakerow"
curl -vi -X PUT \
-H "Accept: text/json" \
-H "Content-Type: text/json" \
-d '{"Row":[{"key":"cm93NQo=", "Cell": [{"column":"Y2Y6ZQo=", "$":"dmFsdWU1Cg=="}]}]}'' \
"example.com:8000/users/fakerow"</pre></div></td>
</tr>
</tbody>
</table>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 19. Endpoints for <code>Check-And-Put</code> Operations</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em>/?check=put</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>PUT</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Conditional Put - Change the current version value of a cell: Compare the current or latest version value (<code>current-version-value</code>) of a cell with the <code>check-value</code>, and if <code>current-version-value</code> == <code>check-value</code>, write new data (the <code>new-value</code>) into the cell as the current or latest version. The row, column qualifier, and value must each be Base-64 encoded. To encode a string, use the <code>base64</code> command-line utility. To decode the string, use <code>base64 -d</code>. The payload is in the <code>--data</code> or <code>-d</code> argument, with <code>the check cell name (column family:column name) and value</code> always at the end and right after <code>the new Put cell name (column family:column name) and value</code> of the same row key. You can also save the data to be inserted to a file and pass it to the <code>-d</code> parameter with syntax like <code>-d @filename.txt</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X PUT \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93MQ=="&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;T2xkR3V5&lt;/Cell&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;TmV3R3V5&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/row1/?check=put"
curl -vi -X PUT \
-H "Accept: application/json" \
-H "Content-Type: application/json" \
-d '{"Row":[{"key":"cm93MQ==","Cell":[{"column":"Y2ZhOmFsaWFz","$":"T2xkR3V5"},{"column":"Y2ZhOmFsaWFz", "$":"TmV3R3V5"}] }]}' \
"http://example.com:8000/users/row1/?check=put"</pre></div></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Detailed Explanation:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>In the above json-format example:</p>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><code>{"column":"Y2ZhOmFsaWFz", "$":"TmV3R3V5"}</code> at the end of <code>-d</code> option are <code>the check cell name and check cell value in Base-64</code> respectively: <code>"Y2ZhOmFsaWFz" for "cfa:alias"</code>, and <code>"TmV3R3V5" for "NewGuy"</code></p>
</li>
<li>
<p><code>{"column":"Y2ZhOmFsaWFz","$":"T2xkR3V5"}</code> are <code>the new Put cell name and cell value in Base-64</code> respectively: <code>"Y2ZhOmFsaWFz" for "cfa:alias"</code>, and <code>"T2xkR3V5" for "OldGuy"</code></p>
</li>
<li>
<p><code>"cm93MQ=="</code> is <code>the Base-64 for "row1"</code> for the checkAndPut <code>row key</code></p>
</li>
<li>
<p><code>"/?check=put"</code> after the <code>"row key" in the request URL</code> is required for checkAndPut WebHBase operation to work</p>
</li>
<li>
<p>The <code>"row key" in the request URL</code> should be URL-encoded, e.g., <code>"david%20chen"</code> and <code>"row1"</code> are the URL-encoded formats of row keys <code>"david chen"</code> and <code>"row1"</code>, respectively</p>
<div class="literalblock">
<div class="content">
<pre>Note: "cfa" is the column family name and "alias" are the column (qualifier) name for the non-Base64 encoded cell name.</pre>
</div>
</div>
</li>
</ol>
</div>
</li>
<li>
<p>Basically, the xml-format example is the same as the json-format example, and will not be explained here in detail.</p>
</li>
</ul>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 20. Endpoints for <code>Check-And-Delete</code> Operations</caption>
<colgroup>
<col style="width: 16.6666%;">
<col style="width: 8.3333%;">
<col style="width: 25%;">
<col style="width: 50.0001%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Endpoint</th>
<th class="tableblock halign-left valign-top">HTTP Verb</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Example</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em>/?check=delete</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Conditional Deleting a Row: Compare the value of any version of a cell (<code>any-version-value</code>) with the <code>check-value</code>, and if <code>any-version-value</code> == <code>check-value</code>, delete the row specified by the <code>row_key</code> inside the requesting URL.The row, column qualifier, and value for checking in the payload must each be Base-64 encoded. To encode a string, use the base64 command-line utility. To decode the string, use base64 -d. The payload is in the --data argument. You can also save the data to be checked to a file and pass it to the <code>-d</code> parameter with syntax like <code>-d @filename.txt</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93MQ=="&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;TmV3R3V5&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/row1/?check=delete"
curl -vi -X DELETE \
-H "Accept: application/json" \
-H "Content-Type: application/json" \
-d '{"Row":[{"key":"cm93MQ==","Cell":[{"column":"Y2ZhOmFsaWFz","$":"TmV3R3V5"}]}]}' \
"http://example.com:8000/users/row1/?check=delete"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em>
/<em>column_family</em>
/?check=delete</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Conditional Deleting a Column Family of a Row: Compare the value of any version of a cell (<code>any-version-value</code>) with the <code>check-value</code>, and if <code>any-version-value</code> == <code>check-value</code>, delete the column family of a row specified by the <code>row_key/column_family</code> inside the requesting URL. Anything else is the same as those in <code>Conditional Deleting a Row</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93MQ=="&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;TmV3R3V5&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/row1/cfa/?check=delete"
curl -vi -X DELETE \
-H "Accept: application/json" \
-H "Content-Type: application/json" \
-d '{"Row":[{"key":"cm93MQ==","Cell":[{"column":"Y2ZhOmFsaWFz","$":"TmV3R3V5"}]}]}' \
"http://example.com:8000/users/row1/cfa/?check=delete"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em>
/<em>column:qualifier</em>
/?check=delete</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Conditional Deleting All Versions of a Column of a Row: Compare the value of any version of a cell (<code>any-version-value</code>) with the <code>check-value</code>, and if <code>any-version-value</code> == <code>check-value</code>, delete the column of a row specified by the <code>row_key/column:qualifier</code> inside the requesting URL. The <code>column:qualifier</code> in the requesting URL is the <code>column_family:column_name</code>. Anything else is the same as those in <code>Conditional Deleting a Row</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93MQ=="&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;TmV3R3V5&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/row1/cfa:alias/?check=delete"
curl -vi -X DELETE \
-H "Accept: application/json" \
-H "Content-Type: application/json" \
-d '{"Row":[{"key":"cm93MQ==","Cell":[{"column":"Y2ZhOmFsaWFz","$":"TmV3R3V5"}]}]}' \
"http://example.com:8000/users/row1/cfa:alias/?check=delete"</pre></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>/<em>table</em>/<em>row_key</em>
/<em>column:qualifier</em>
/<em>version_id</em>/?check=delete</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>DELETE</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Conditional Deleting a Single Version of a Column of a Row: Compare the value of any version of a cell (<code>any-version-value</code>) with the <code>check-value</code>, and if <code>any-version-value</code> == <code>check-value</code>, delete the version of a column of a row specified by the <code>row_key/column:qualifier/version_id</code> inside the requesting URL. The <code>column:qualifier</code> in the requesting URL is the <code>column_family:column_name</code>. The <code>version_id</code> in the requesting URL is a number, which equals to <code>the timestamp of the targeted version + 1</code>. Anything else is the same as those in <code>Conditional Deleting a Row</code>.</p></td>
<td class="tableblock halign-left valign-top"><div class="literal"><pre>curl -vi -X DELETE \
-H "Accept: text/xml" \
-H "Content-Type: text/xml" \
-d '&lt;?xml version="1.0" encoding="UTF-8" standalone="yes"?&gt;&lt;CellSet&gt;&lt;Row key="cm93MQ=="&gt;&lt;Cell column="Y2ZhOmFsaWFz"&gt;TmV3R3V5&lt;/Cell&gt;&lt;/Row&gt;&lt;/CellSet&gt;' \
"http://example.com:8000/users/row1/cfa:alias/1519423552160/?check=delete"
curl -vi -X DELETE \
-H "Accept: application/json" \
-H "Content-Type: application/json" \
-d '{"Row":[{"key":"cm93MQ==","Cell":[{"column":"Y2ZhOmFsaWFz","$":"TmV3R3V5"}]}]}' \
"http://example.com:8000/users/row1/cfa:alias/1519423552160/?check=delete"</pre></div></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Detailed Explanation:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>In the above 4 json-format examples:</p>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><code>{"column":"Y2ZhOmFsaWFz", "$":"TmV3R3V5"}</code> at the end of <code>-d</code> option are <code>the check cell name and check cell value in Base-64</code> respectively: <code>"Y2ZhOmFsaWFz" for "cfa:alias"</code>, and <code>"TmV3R3V5" for "NewGuy"</code></p>
</li>
<li>
<p><code>"cm93MQ=="</code> is <code>the Base-64 for "row1"</code> for the checkAndDelete <code>row key</code></p>
</li>
<li>
<p><code>"/?check=delete"</code> at the end of <code>the request URL</code> is required for checkAndDelete WebHBase operation to work</p>
</li>
<li>
<p><code>"version_id"</code> in the <code>request URL</code> of the last json-format example should be equivalent to the value of <code>"the timestamp number + 1"</code></p>
</li>
<li>
<p>The <code>"row key"</code>, <code>"column family"</code>, <code>"cell name" or "column family:column name"</code>, and <code>"version_id"</code> in <code>the request URL</code> of a checkAndDelete WebHBase operation should be URL-encoded, e.g., <code>"row1"</code>, <code>"cfa"</code>, <code>"cfa:alias"</code> and <code>"1519423552160"</code> in the examples are the URL-encoded <code>"row key"</code>, <code>"column family"</code>, <code>"column family:column name"</code>, and <code>"version_id"</code>, respectively</p>
</li>
</ol>
</div>
</li>
<li>
<p>Basically, the 4 xml-format examples are the same as the 4 corresponding json-format examples, and will not be explained here in detail.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="xml_schema"><a class="anchor" href="#xml_schema"></a>101.4. REST XML Schema</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;schema</span> <span class="attribute-name">xmlns</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">http://www.w3.org/2001/XMLSchema</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">xmlns:tns</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">RESTSchema</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Version</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Version</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Version</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">REST</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">JVM</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">OS</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Server</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Jersey</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableList</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:TableList</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableList</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">table</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Table</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Table</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableInfo</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:TableInfo</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableInfo</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">region</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:TableRegion</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableRegion</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">startKey</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">endKey</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">location</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableSchema</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:TableSchema</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">TableSchema</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">column</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:ColumnSchema</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;anyAttribute&gt;</span><span class="tag">&lt;/anyAttribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">ColumnSchema</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;anyAttribute&gt;</span><span class="tag">&lt;/anyAttribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">CellSet</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:CellSet</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">CellSet</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">row</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Row</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Row</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Row</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Row</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">key</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">cell</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Cell</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Cell</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Cell</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Cell</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">value</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;simpleType&gt;</span><span class="tag">&lt;restriction</span> <span class="attribute-name">base</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;/simpleType&gt;</span>
<span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">column</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span> <span class="tag">/&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">timestamp</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span> <span class="tag">/&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Scanner</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Scanner</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Scanner</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">column</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">filter</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">startRow</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">endRow</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">batch</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">startTime</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">endTime</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">StorageClusterVersion</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:StorageClusterVersion</span><span class="delimiter">&quot;</span></span> <span class="tag">/&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">StorageClusterVersion</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">version</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">StorageClusterStatus</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:StorageClusterStatus</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">StorageClusterStatus</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">liveNode</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Node</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">deadNode</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">regions</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">requests</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">averageLoad</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">float</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Node</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;sequence&gt;</span>
<span class="tag">&lt;element</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">region</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">tns:Region</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">maxOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">unbounded</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">minOccurs</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;/element&gt;</span>
<span class="tag">&lt;/sequence&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">string</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">startCode</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">requests</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">heapSizeMB</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">maxHeapSizeMB</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;complexType</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">Region</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">base64Binary</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">stores</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">storefiles</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">storefileSizeMB</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">memstoreSizeMB</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;attribute</span> <span class="attribute-name">name</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">storefileIndexSizeMB</span><span class="delimiter">&quot;</span></span> <span class="attribute-name">type</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">int</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span><span class="tag">&lt;/attribute&gt;</span>
<span class="tag">&lt;/complexType&gt;</span>
<span class="tag">&lt;/schema&gt;</span></code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="protobufs_schema"><a class="anchor" href="#protobufs_schema"></a>101.5. REST Protobufs Schema</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="json"><span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> {
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">r</span><span class="error">e</span><span class="error">s</span><span class="error">t</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">j</span><span class="error">v</span><span class="error">m</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">o</span><span class="error">s</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">s</span><span class="error">e</span><span class="error">r</span><span class="error">v</span><span class="error">e</span><span class="error">r</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">j</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">e</span><span class="error">y</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">S</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">a</span><span class="error">g</span><span class="error">e</span><span class="error">C</span><span class="error">l</span><span class="error">u</span><span class="error">s</span><span class="error">t</span><span class="error">e</span><span class="error">r</span><span class="error">S</span><span class="error">t</span><span class="error">a</span><span class="error">t</span><span class="error">u</span><span class="error">s</span> {
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">R</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">s</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">e</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">s</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">e</span><span class="error">f</span><span class="error">i</span><span class="error">l</span><span class="error">e</span><span class="error">s</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">s</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">e</span><span class="error">f</span><span class="error">i</span><span class="error">l</span><span class="error">e</span><span class="error">S</span><span class="error">i</span><span class="error">z</span><span class="error">e</span><span class="error">M</span><span class="error">B</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">m</span><span class="error">e</span><span class="error">m</span><span class="error">s</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">e</span><span class="error">S</span><span class="error">i</span><span class="error">z</span><span class="error">e</span><span class="error">M</span><span class="error">B</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">s</span><span class="error">t</span><span class="error">o</span><span class="error">r</span><span class="error">e</span><span class="error">f</span><span class="error">i</span><span class="error">l</span><span class="error">e</span><span class="error">I</span><span class="error">n</span><span class="error">d</span><span class="error">e</span><span class="error">x</span><span class="error">S</span><span class="error">i</span><span class="error">z</span><span class="error">e</span><span class="error">M</span><span class="error">B</span> <span class="error">=</span> <span class="integer">6</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">N</span><span class="error">o</span><span class="error">d</span><span class="error">e</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span> <span class="error">/</span><span class="error">/</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span>:<span class="error">p</span><span class="error">o</span><span class="error">r</span><span class="error">t</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">64</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">r</span><span class="error">t</span><span class="error">C</span><span class="error">o</span><span class="error">d</span><span class="error">e</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">e</span><span class="error">s</span><span class="error">t</span><span class="error">s</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">h</span><span class="error">e</span><span class="error">a</span><span class="error">p</span><span class="error">S</span><span class="error">i</span><span class="error">z</span><span class="error">e</span><span class="error">M</span><span class="error">B</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">m</span><span class="error">a</span><span class="error">x</span><span class="error">H</span><span class="error">e</span><span class="error">a</span><span class="error">p</span><span class="error">S</span><span class="error">i</span><span class="error">z</span><span class="error">e</span><span class="error">M</span><span class="error">B</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">R</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">r</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">6</span><span class="error">;</span>
}
<span class="error">/</span><span class="error">/</span> <span class="error">n</span><span class="error">o</span><span class="error">d</span><span class="error">e</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">t</span><span class="error">u</span><span class="error">s</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">N</span><span class="error">o</span><span class="error">d</span><span class="error">e</span> <span class="error">l</span><span class="error">i</span><span class="error">v</span><span class="error">e</span><span class="error">N</span><span class="error">o</span><span class="error">d</span><span class="error">e</span><span class="error">s</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">d</span><span class="error">e</span><span class="error">a</span><span class="error">d</span><span class="error">N</span><span class="error">o</span><span class="error">d</span><span class="error">e</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">/</span><span class="error">/</span> <span class="error">s</span><span class="error">u</span><span class="error">m</span><span class="error">m</span><span class="error">a</span><span class="error">r</span><span class="error">y</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">t</span><span class="error">i</span><span class="error">s</span><span class="error">t</span><span class="error">i</span><span class="error">c</span><span class="error">s</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">r</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">e</span><span class="error">s</span><span class="error">t</span><span class="error">s</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">d</span><span class="error">o</span><span class="error">u</span><span class="error">b</span><span class="error">l</span><span class="error">e</span> <span class="error">a</span><span class="error">v</span><span class="error">e</span><span class="error">r</span><span class="error">a</span><span class="error">g</span><span class="error">e</span><span class="error">L</span><span class="error">o</span><span class="error">a</span><span class="error">d</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">T</span><span class="error">a</span><span class="error">b</span><span class="error">l</span><span class="error">e</span><span class="error">L</span><span class="error">i</span><span class="error">s</span><span class="error">t</span> {
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">T</span><span class="error">a</span><span class="error">b</span><span class="error">l</span><span class="error">e</span><span class="error">I</span><span class="error">n</span><span class="error">f</span><span class="error">o</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">R</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">r</span><span class="error">t</span><span class="error">K</span><span class="error">e</span><span class="error">y</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">e</span><span class="error">n</span><span class="error">d</span><span class="error">K</span><span class="error">e</span><span class="error">y</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">64</span> <span class="error">i</span><span class="error">d</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">l</span><span class="error">o</span><span class="error">c</span><span class="error">a</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
}
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">R</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">r</span><span class="error">e</span><span class="error">g</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">T</span><span class="error">a</span><span class="error">b</span><span class="error">l</span><span class="error">e</span><span class="error">S</span><span class="error">c</span><span class="error">h</span><span class="error">e</span><span class="error">m</span><span class="error">a</span> {
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">A</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">v</span><span class="error">a</span><span class="error">l</span><span class="error">u</span><span class="error">e</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
}
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">A</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span> <span class="error">a</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">C</span><span class="error">o</span><span class="error">l</span><span class="error">u</span><span class="error">m</span><span class="error">n</span><span class="error">S</span><span class="error">c</span><span class="error">h</span><span class="error">e</span><span class="error">m</span><span class="error">a</span> <span class="error">c</span><span class="error">o</span><span class="error">l</span><span class="error">u</span><span class="error">m</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">/</span><span class="error">/</span> <span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">h</span><span class="error">e</span><span class="error">l</span><span class="error">p</span><span class="error">f</span><span class="error">u</span><span class="error">l</span> <span class="error">e</span><span class="error">n</span><span class="error">c</span><span class="error">o</span><span class="error">d</span><span class="error">i</span><span class="error">n</span><span class="error">g</span><span class="error">s</span> <span class="error">o</span><span class="error">f</span> <span class="error">c</span><span class="error">o</span><span class="error">m</span><span class="error">m</span><span class="error">o</span><span class="error">n</span><span class="error">l</span><span class="error">y</span> <span class="error">u</span><span class="error">s</span><span class="error">e</span><span class="error">d</span> <span class="error">a</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span><span class="error">s</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">o</span><span class="error">o</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">M</span><span class="error">e</span><span class="error">m</span><span class="error">o</span><span class="error">r</span><span class="error">y</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">o</span><span class="error">o</span><span class="error">l</span> <span class="error">r</span><span class="error">e</span><span class="error">a</span><span class="error">d</span><span class="error">O</span><span class="error">n</span><span class="error">l</span><span class="error">y</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">C</span><span class="error">o</span><span class="error">l</span><span class="error">u</span><span class="error">m</span><span class="error">n</span><span class="error">S</span><span class="error">c</span><span class="error">h</span><span class="error">e</span><span class="error">m</span><span class="error">a</span> {
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">A</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">n</span><span class="error">a</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">v</span><span class="error">a</span><span class="error">l</span><span class="error">u</span><span class="error">e</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
}
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">A</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span> <span class="error">a</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">/</span><span class="error">/</span> <span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">h</span><span class="error">e</span><span class="error">l</span><span class="error">p</span><span class="error">f</span><span class="error">u</span><span class="error">l</span> <span class="error">e</span><span class="error">n</span><span class="error">c</span><span class="error">o</span><span class="error">d</span><span class="error">i</span><span class="error">n</span><span class="error">g</span><span class="error">s</span> <span class="error">o</span><span class="error">f</span> <span class="error">c</span><span class="error">o</span><span class="error">m</span><span class="error">m</span><span class="error">o</span><span class="error">n</span><span class="error">l</span><span class="error">y</span> <span class="error">u</span><span class="error">s</span><span class="error">e</span><span class="error">d</span> <span class="error">a</span><span class="error">t</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">b</span><span class="error">u</span><span class="error">t</span><span class="error">e</span><span class="error">s</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">t</span><span class="error">t</span><span class="error">l</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">m</span><span class="error">a</span><span class="error">x</span><span class="error">V</span><span class="error">e</span><span class="error">r</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">s</span><span class="error">t</span><span class="error">r</span><span class="error">i</span><span class="error">n</span><span class="error">g</span> <span class="error">c</span><span class="error">o</span><span class="error">m</span><span class="error">p</span><span class="error">r</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">i</span><span class="error">o</span><span class="error">n</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">C</span><span class="error">e</span><span class="error">l</span><span class="error">l</span> {
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">r</span><span class="error">o</span><span class="error">w</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span> <span class="error">/</span><span class="error">/</span> <span class="error">u</span><span class="error">n</span><span class="error">u</span><span class="error">s</span><span class="error">e</span><span class="error">d</span> <span class="error">i</span><span class="error">f</span> <span class="error">C</span><span class="error">e</span><span class="error">l</span><span class="error">l</span> <span class="error">i</span><span class="error">s</span> <span class="error">i</span><span class="error">n</span> <span class="error">a</span> <span class="error">C</span><span class="error">e</span><span class="error">l</span><span class="error">l</span><span class="error">S</span><span class="error">e</span><span class="error">t</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">c</span><span class="error">o</span><span class="error">l</span><span class="error">u</span><span class="error">m</span><span class="error">n</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">64</span> <span class="error">t</span><span class="error">i</span><span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">m</span><span class="error">p</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">d</span><span class="error">a</span><span class="error">t</span><span class="error">a</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">C</span><span class="error">e</span><span class="error">l</span><span class="error">l</span><span class="error">S</span><span class="error">e</span><span class="error">t</span> {
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">R</span><span class="error">o</span><span class="error">w</span> {
<span class="error">r</span><span class="error">e</span><span class="error">q</span><span class="error">u</span><span class="error">i</span><span class="error">r</span><span class="error">e</span><span class="error">d</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">k</span><span class="error">e</span><span class="error">y</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">C</span><span class="error">e</span><span class="error">l</span><span class="error">l</span> <span class="error">v</span><span class="error">a</span><span class="error">l</span><span class="error">u</span><span class="error">e</span><span class="error">s</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
}
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">R</span><span class="error">o</span><span class="error">w</span> <span class="error">r</span><span class="error">o</span><span class="error">w</span><span class="error">s</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
}
<span class="error">m</span><span class="error">e</span><span class="error">s</span><span class="error">s</span><span class="error">a</span><span class="error">g</span><span class="error">e</span> <span class="error">S</span><span class="error">c</span><span class="error">a</span><span class="error">n</span><span class="error">n</span><span class="error">e</span><span class="error">r</span> {
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">r</span><span class="error">t</span><span class="error">R</span><span class="error">o</span><span class="error">w</span> <span class="error">=</span> <span class="integer">1</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">e</span><span class="error">n</span><span class="error">d</span><span class="error">R</span><span class="error">o</span><span class="error">w</span> <span class="error">=</span> <span class="integer">2</span><span class="error">;</span>
<span class="error">r</span><span class="error">e</span><span class="error">p</span><span class="error">e</span><span class="error">a</span><span class="error">t</span><span class="error">e</span><span class="error">d</span> <span class="error">b</span><span class="error">y</span><span class="error">t</span><span class="error">e</span><span class="error">s</span> <span class="error">c</span><span class="error">o</span><span class="error">l</span><span class="error">u</span><span class="error">m</span><span class="error">n</span><span class="error">s</span> <span class="error">=</span> <span class="integer">3</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">32</span> <span class="error">b</span><span class="error">a</span><span class="error">t</span><span class="error">c</span><span class="error">h</span> <span class="error">=</span> <span class="integer">4</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">64</span> <span class="error">s</span><span class="error">t</span><span class="error">a</span><span class="error">r</span><span class="error">t</span><span class="error">T</span><span class="error">i</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">5</span><span class="error">;</span>
<span class="error">o</span><span class="error">p</span><span class="error">t</span><span class="error">i</span><span class="error">o</span><span class="error">n</span><span class="error">a</span><span class="error">l</span> <span class="error">i</span><span class="error">n</span><span class="error">t</span><span class="integer">64</span> <span class="error">e</span><span class="error">n</span><span class="error">d</span><span class="error">T</span><span class="error">i</span><span class="error">m</span><span class="error">e</span> <span class="error">=</span> <span class="integer">6</span><span class="error">;</span>
}</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_thrift"><a class="anchor" href="#_thrift"></a>102. Thrift</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Documentation about Thrift has moved to <a href="#thrift">Thrift API and Filter Language</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="c"><a class="anchor" href="#c"></a>103. C/C++ Apache HBase Client</h2>
<div class="sectionbody">
<div class="paragraph">
<p>FB&#8217;s Chip Turner wrote a pure C/C++ client.
<a href="https://github.com/hinaria/native-cpp-hbase-client">Check it out</a>.</p>
</div>
<div class="paragraph">
<p>C++ client implementation. To see <a href="https://issues.apache.org/jira/browse/HBASE-14850">HBASE-14850</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="jdo"><a class="anchor" href="#jdo"></a>104. Using Java Data Objects (JDO) with HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="https://db.apache.org/jdo/">Java Data Objects (JDO)</a> is a standard way to
access persistent data in databases, using plain old Java objects (POJO) to
represent persistent data.</p>
</div>
<div class="paragraph">
<div class="title">Dependencies</div>
<p>This code example has the following dependencies:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>HBase 0.90.x or newer</p>
</li>
<li>
<p>commons-beanutils.jar (<a href="https://commons.apache.org/" class="bare">https://commons.apache.org/</a>)</p>
</li>
<li>
<p>commons-pool-1.5.5.jar (<a href="https://commons.apache.org/" class="bare">https://commons.apache.org/</a>)</p>
</li>
<li>
<p>transactional-tableindexed for HBase 0.90 (<a href="https://github.com/hbase-trx/hbase-transactional-tableindexed" class="bare">https://github.com/hbase-trx/hbase-transactional-tableindexed</a>)</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Download <code>hbase-jdo</code></div>
<p>Download the code from <a href="http://code.google.com/p/hbase-jdo/" class="bare">http://code.google.com/p/hbase-jdo/</a>.</p>
</div>
<div class="exampleblock">
<div class="title">Example 29. JDO Example</div>
<div class="content">
<div class="paragraph">
<p>This example uses JDO to create a table and an index, insert a row into a table, get
a row, get a column value, perform a query, and do some additional HBase operations.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">package</span> <span class="namespace">com.apache.hadoop.hbase.client.jdo.examples</span>;
<span class="keyword">import</span> <span class="include">java.io.File</span>;
<span class="keyword">import</span> <span class="include">java.io.FileInputStream</span>;
<span class="keyword">import</span> <span class="include">java.io.InputStream</span>;
<span class="keyword">import</span> <span class="include">java.util.Hashtable</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.fs.Path</span>;
<span class="keyword">import</span> <span class="include">org.apache.hadoop.hbase.client.tableindexed.IndexedTable</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.AbstractHBaseDBO</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.HBaseBigFile</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.HBaseDBOImpl</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.DeleteQuery</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.HBaseOrder</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.HBaseParam</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.InsertQuery</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.QSearch</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.SelectQuery</span>;
<span class="keyword">import</span> <span class="include">com.apache.hadoop.hbase.client.jdo.query.UpdateQuery</span>;
<span class="comment">/**
* Hbase JDO Example.
*
* dependency library.
* - commons-beanutils.jar
* - commons-pool-1.5.5.jar
* - hbase0.90.0-transactionl.jar
*
* you can expand Delete,Select,Update,Insert Query classes.
*
*/</span>
<span class="directive">public</span> <span class="type">class</span> <span class="class">HBaseExample</span> {
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> main(<span class="predefined-type">String</span><span class="type">[]</span> args) <span class="directive">throws</span> <span class="exception">Exception</span> {
AbstractHBaseDBO dbo = <span class="keyword">new</span> HBaseDBOImpl();
<span class="comment">//*drop if table is already exist.*</span>
<span class="keyword">if</span>(dbo.isTableExist(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>)){
dbo.deleteTable(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
}
<span class="comment">//*create table*</span>
dbo.createTableIfNotExist(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>,HBaseOrder.DESC,<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>);
<span class="comment">//dbo.createTableIfNotExist(&quot;user&quot;,HBaseOrder.ASC,&quot;account&quot;);</span>
<span class="comment">//create index.</span>
<span class="predefined-type">String</span><span class="type">[]</span> cols={<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span>};
dbo.addIndexExistingTable(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>,cols);
<span class="comment">//insert</span>
InsertQuery insert = dbo.createInsertQuery(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
UserBean bean = <span class="keyword">new</span> UserBean();
bean.setFamily(<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>);
bean.setAge(<span class="integer">20</span>);
bean.setEmail(<span class="string"><span class="delimiter">&quot;</span><span class="content">ncanis@gmail.com</span><span class="delimiter">&quot;</span></span>);
bean.setId(<span class="string"><span class="delimiter">&quot;</span><span class="content">ncanis</span><span class="delimiter">&quot;</span></span>);
bean.setName(<span class="string"><span class="delimiter">&quot;</span><span class="content">ncanis</span><span class="delimiter">&quot;</span></span>);
bean.setPassword(<span class="string"><span class="delimiter">&quot;</span><span class="content">1111</span><span class="delimiter">&quot;</span></span>);
insert.insert(bean);
<span class="comment">//select 1 row</span>
SelectQuery select = dbo.createSelectQuery(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
UserBean resultBean = (UserBean)select.select(bean.getRow(),UserBean.class);
<span class="comment">// select column value.</span>
<span class="predefined-type">String</span> value = (<span class="predefined-type">String</span>)select.selectColumn(bean.getRow(),<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span>,<span class="predefined-type">String</span>.class);
<span class="comment">// search with option (QSearch has EQUAL, NOT_EQUAL, LIKE)</span>
<span class="comment">// select id,password,name,email from account where id='ncanis' limit startRow,20</span>
HBaseParam param = <span class="keyword">new</span> HBaseParam();
param.setPage(bean.getRow(),<span class="integer">20</span>);
param.addColumn(<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">password</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">name</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">email</span><span class="delimiter">&quot;</span></span>);
param.addSearchOption(<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">ncanis</span><span class="delimiter">&quot;</span></span>,QSearch.EQUAL);
select.search(<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>, param, UserBean.class);
<span class="comment">// search column value is existing.</span>
<span class="type">boolean</span> isExist = select.existColumnValue(<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">id</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">ncanis</span><span class="delimiter">&quot;</span></span>.getBytes());
<span class="comment">// update password.</span>
UpdateQuery update = dbo.createUpdateQuery(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">Hashtable</span>&lt;<span class="predefined-type">String</span>, <span class="type">byte</span><span class="type">[]</span>&gt; colsTable = <span class="keyword">new</span> <span class="predefined-type">Hashtable</span>&lt;<span class="predefined-type">String</span>, <span class="type">byte</span><span class="type">[]</span>&gt;();
colsTable.put(<span class="string"><span class="delimiter">&quot;</span><span class="content">password</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">2222</span><span class="delimiter">&quot;</span></span>.getBytes());
update.update(bean.getRow(),<span class="string"><span class="delimiter">&quot;</span><span class="content">account</span><span class="delimiter">&quot;</span></span>,colsTable);
<span class="comment">//delete</span>
DeleteQuery delete = dbo.createDeleteQuery(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
delete.deleteRow(resultBean.getRow());
<span class="comment">////////////////////////////////////</span>
<span class="comment">// etc</span>
<span class="comment">// HTable pool with apache commons pool</span>
<span class="comment">// borrow and release. HBasePoolManager(maxActive, minIdle etc..)</span>
IndexedTable table = dbo.getPool().borrow(<span class="string"><span class="delimiter">&quot;</span><span class="content">user</span><span class="delimiter">&quot;</span></span>);
dbo.getPool().release(table);
<span class="comment">// upload bigFile by hadoop directly.</span>
HBaseBigFile bigFile = <span class="keyword">new</span> HBaseBigFile();
<span class="predefined-type">File</span> file = <span class="keyword">new</span> <span class="predefined-type">File</span>(<span class="string"><span class="delimiter">&quot;</span><span class="content">doc/movie.avi</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">FileInputStream</span> fis = <span class="keyword">new</span> <span class="predefined-type">FileInputStream</span>(file);
Path rootPath = <span class="keyword">new</span> Path(<span class="string"><span class="delimiter">&quot;</span><span class="content">/files/</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">String</span> filename = <span class="string"><span class="delimiter">&quot;</span><span class="content">movie.avi</span><span class="delimiter">&quot;</span></span>;
bigFile.uploadFile(rootPath,filename,fis,<span class="predefined-constant">true</span>);
<span class="comment">// receive file stream from hadoop.</span>
Path p = <span class="keyword">new</span> Path(rootPath,filename);
<span class="predefined-type">InputStream</span> is = bigFile.path2Stream(p,<span class="integer">4096</span>);
}
}</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="scala"><a class="anchor" href="#scala"></a>105. Scala</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_setting_the_classpath"><a class="anchor" href="#_setting_the_classpath"></a>105.1. Setting the Classpath</h3>
<div class="paragraph">
<p>To use Scala with HBase, your CLASSPATH must include HBase&#8217;s classpath as well as
the Scala JARs required by your code. First, use the following command on a server
running the HBase RegionServer process, to get HBase&#8217;s classpath.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ ps aux |grep regionserver| awk -F 'java.library.path=' {'print $2'} | awk {'print $1'}
/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64</code></pre>
</div>
</div>
<div class="paragraph">
<p>Set the <code>$CLASSPATH</code> environment variable to include the path you found in the previous
step, plus the path of <code>scala-library.jar</code> and each additional Scala-related JAR needed for
your project.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ export CLASSPATH=$CLASSPATH:/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64:/path/to/scala-library.jar</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_scala_sbt_file"><a class="anchor" href="#_scala_sbt_file"></a>105.2. Scala SBT File</h3>
<div class="paragraph">
<p>Your <code>build.sbt</code> file needs the following <code>resolvers</code> and <code>libraryDependencies</code> to work
with HBase.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>resolvers += "Apache HBase" at "https://repository.apache.org/content/repositories/releases"
resolvers += "Thrift" at "https://people.apache.org/~rawson/repo/"
libraryDependencies ++= Seq(
"org.apache.hadoop" % "hadoop-core" % "0.20.2",
"org.apache.hbase" % "hbase" % "0.90.4"
)</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_example_scala_code"><a class="anchor" href="#_example_scala_code"></a>105.3. Example Scala Code</h3>
<div class="paragraph">
<p>This example lists HBase tables, creates a new table, and adds a row to it.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">import org.apache.hadoop.hbase.HBaseConfiguration
import org.apache.hadoop.hbase.client.{Connection,ConnectionFactory,HBaseAdmin,HTable,Put,Get}
import org.apache.hadoop.hbase.util.Bytes
val conf = new HBaseConfiguration()
val connection = ConnectionFactory.createConnection(conf);
val admin = connection.getAdmin();
// list the tables
val listtables=admin.listTables()
listtables.foreach(println)
// let's insert some data in 'mytable' and get the row
val table = new HTable(conf, &quot;mytable&quot;)
val theput= new Put(Bytes.toBytes(&quot;rowkey1&quot;))
theput.add(Bytes.toBytes(&quot;ids&quot;),Bytes.toBytes(&quot;id1&quot;),Bytes.toBytes(&quot;one&quot;))
table.put(theput)
val theget= new Get(Bytes.toBytes(&quot;rowkey1&quot;))
val result=table.get(theget)
val value=result.value()
println(Bytes.toString(value))</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="jython"><a class="anchor" href="#jython"></a>106. Jython</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_setting_the_classpath_2"><a class="anchor" href="#_setting_the_classpath_2"></a>106.1. Setting the Classpath</h3>
<div class="paragraph">
<p>To use Jython with HBase, your CLASSPATH must include HBase&#8217;s classpath as well as
the Jython JARs required by your code.</p>
</div>
<div class="paragraph">
<p>Set the path to directory containing the <code>jython.jar</code> and each additional Jython-related JAR needed for
your project. Then export HBASE_CLASSPATH pointing to the $JYTHON_HOME env. variable.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ export HBASE_CLASSPATH=/directory/jython.jar</code></pre>
</div>
</div>
<div class="paragraph">
<p>Start a Jython shell with HBase and Hadoop JARs in the classpath:
$ bin/hbase org.python.util.jython</p>
</div>
</div>
<div class="sect2">
<h3 id="_jython_code_examples"><a class="anchor" href="#_jython_code_examples"></a>106.2. Jython Code Examples</h3>
<div class="exampleblock">
<div class="title">Example 30. Table Creation, Population, Get, and Delete with Jython</div>
<div class="content">
<div class="paragraph">
<p>The following Jython code example checks for table,
if it exists, deletes it and then creates it. Then it
populates the table with data and fetches the data.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="jython">import java.lang
from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, TableName
from org.apache.hadoop.hbase.client import Admin, Connection, ConnectionFactory, Get, Put, Result, Table
from org.apache.hadoop.conf import Configuration
# First get a conf object. This will read in the configuration
# that is out in your hbase-*.xml files such as location of the
# hbase master node.
conf = HBaseConfiguration.create()
connection = ConnectionFactory.createConnection(conf)
admin = connection.getAdmin()
# Create a table named 'test' that has a column family
# named 'content'.
tableName = TableName.valueOf(&quot;test&quot;)
table = connection.getTable(tableName)
desc = HTableDescriptor(tableName)
desc.addFamily(HColumnDescriptor(&quot;content&quot;))
# Drop and recreate if it exists
if admin.tableExists(tableName):
admin.disableTable(tableName)
admin.deleteTable(tableName)
admin.createTable(desc)
# Add content to 'column:' on a row named 'row_x'
row = 'row_x'
put = Put(row)
put.addColumn(&quot;content&quot;, &quot;qual&quot;, &quot;some content&quot;)
table.put(put)
# Now fetch the content just added, returns a byte[]
get = Get(row)
result = table.get(get)
data = java.lang.String(result.getValue(&quot;content&quot;, &quot;qual&quot;), &quot;UTF8&quot;)
print &quot;The fetched row contains the value '%s'&quot; % data</code></pre>
</div>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 31. Table Scan Using Jython</div>
<div class="content">
<div class="paragraph">
<p>This example scans a table and returns the results that match a given family qualifier.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="jython">import java.lang
from org.apache.hadoop.hbase import TableName, HBaseConfiguration
from org.apache.hadoop.hbase.client import Connection, ConnectionFactory, Result, ResultScanner, Table, Admin
from org.apache.hadoop.conf import Configuration
conf = HBaseConfiguration.create()
connection = ConnectionFactory.createConnection(conf)
admin = connection.getAdmin()
tableName = TableName.valueOf('wiki')
table = connection.getTable(tableName)
cf = &quot;title&quot;
attr = &quot;attr&quot;
scanner = table.getScanner(cf)
while 1:
result = scanner.next()
if not result:
break
print java.lang.String(result.row), java.lang.String(result.getValue(cf, attr))</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
<h1 id="thrift" class="sect0"><a class="anchor" href="#thrift"></a>Thrift API and Filter Language</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>Apache <a href="https://thrift.apache.org/">Thrift</a> is a cross-platform, cross-language development framework.
HBase includes a Thrift API and filter language.
The Thrift API relies on client and server processes.</p>
</div>
<div class="paragraph">
<p>You can configure Thrift for secure authentication at the server and client side, by following the procedures in <a href="#security.client.thrift">Client-side Configuration for Secure Operation - Thrift Gateway</a> and <a href="#security.gateway.thrift">Configure the Thrift Gateway to Authenticate on Behalf of the Client</a>.</p>
</div>
<div class="paragraph">
<p>The rest of this chapter discusses the filter language provided by the Thrift API.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="thrift.filter_language"><a class="anchor" href="#thrift.filter_language"></a>107. Filter Language</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Thrift Filter Language was introduced in HBase 0.92.
It allows you to perform server-side filtering when accessing HBase over Thrift or in the HBase shell.
You can find out more about shell integration by using the <code>scan help</code> command in the shell.</p>
</div>
<div class="paragraph">
<p>You specify a filter as a string, which is parsed on the server to construct the filter.</p>
</div>
<div class="sect2">
<h3 id="general_syntax"><a class="anchor" href="#general_syntax"></a>107.1. General Filter String Syntax</h3>
<div class="paragraph">
<p>A simple filter expression is expressed as a string:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>“FilterName (argument, argument,... , argument)”</pre>
</div>
</div>
<div class="paragraph">
<p>Keep the following syntax guidelines in mind.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Specify the name of the filter followed by the comma-separated argument list in parentheses.</p>
</li>
<li>
<p>If the argument represents a string, it should be enclosed in single quotes (<code>'</code>).</p>
</li>
<li>
<p>Arguments which represent a boolean, an integer, or a comparison operator (such as &lt;, &gt;, or !=), should not be enclosed in quotes</p>
</li>
<li>
<p>The filter name must be a single word.
All ASCII characters are allowed except for whitespace, single quotes and parentheses.</p>
</li>
<li>
<p>The filter&#8217;s arguments can contain any ASCII character.
If single quotes are present in the argument, they must be escaped by an additional preceding single quote.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_compound_filters_and_operators"><a class="anchor" href="#_compound_filters_and_operators"></a>107.2. Compound Filters and Operators</h3>
<div class="dlist">
<div class="title">Binary Operators</div>
<dl>
<dt class="hdlist1"><code>AND</code></dt>
<dd>
<p>If the <code>AND</code> operator is used, the key-value must satisfy both filters.</p>
</dd>
<dt class="hdlist1"><code>OR</code></dt>
<dd>
<p>If the <code>OR</code> operator is used, the key-value must satisfy at least one of the filters.</p>
</dd>
</dl>
</div>
<div class="dlist">
<div class="title">Unary Operators</div>
<dl>
<dt class="hdlist1"><code>SKIP</code></dt>
<dd>
<p>For a particular row, if any of the key-values fail the filter condition, the entire row is skipped.</p>
</dd>
<dt class="hdlist1"><code>WHILE</code></dt>
<dd>
<p>For a particular row, key-values will be emitted until a key-value is reached that fails the filter condition.</p>
</dd>
</dl>
</div>
<div class="exampleblock">
<div class="title">Example 32. Compound Operators</div>
<div class="content">
<div class="paragraph">
<p>You can combine multiple operators to create a hierarchy of filters, such as the following example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">(Filter1 AND Filter2) OR (Filter3 AND Filter4)</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_order_of_evaluation"><a class="anchor" href="#_order_of_evaluation"></a>107.3. Order of Evaluation</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Parentheses have the highest precedence.</p>
</li>
<li>
<p>The unary operators <code>SKIP</code> and <code>WHILE</code> are next, and have the same precedence.</p>
</li>
<li>
<p>The binary operators follow. <code>AND</code> has highest precedence, followed by <code>OR</code>.</p>
</li>
</ol>
</div>
<div class="exampleblock">
<div class="title">Example 33. Precedence Example</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Filter1 AND Filter2 OR <span class="predefined-type">Filter</span>
is evaluated as
(Filter1 AND Filter2) OR Filter3</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Filter1 AND SKIP Filter2 OR Filter3
is evaluated as
(Filter1 AND (SKIP Filter2)) OR Filter3</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>You can use parentheses to explicitly control the order of evaluation.</p>
</div>
</div>
<div class="sect2">
<h3 id="_compare_operator"><a class="anchor" href="#_compare_operator"></a>107.4. Compare Operator</h3>
<div class="paragraph">
<p>The following compare operators are provided:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>LESS (&lt;)</p>
</li>
<li>
<p>LESS_OR_EQUAL (&#8656;)</p>
</li>
<li>
<p>EQUAL (=)</p>
</li>
<li>
<p>NOT_EQUAL (!=)</p>
</li>
<li>
<p>GREATER_OR_EQUAL (&gt;=)</p>
</li>
<li>
<p>GREATER (&gt;)</p>
</li>
<li>
<p>NO_OP (no operation)</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The client should use the symbols (&lt;, &#8656;, =, !=, &gt;, &gt;=) to express compare operators.</p>
</div>
</div>
<div class="sect2">
<h3 id="_comparator"><a class="anchor" href="#_comparator"></a>107.5. Comparator</h3>
<div class="paragraph">
<p>A comparator can be any of the following:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><em>BinaryComparator</em> - This lexicographically compares against the specified byte array using Bytes.compareTo(byte[], byte[])</p>
</li>
<li>
<p><em>BinaryPrefixComparator</em> - This lexicographically compares against a specified byte array.
It only compares up to the length of this byte array.</p>
</li>
<li>
<p><em>RegexStringComparator</em> - This compares against the specified byte array using the given regular expression.
Only EQUAL and NOT_EQUAL comparisons are valid with this comparator</p>
</li>
<li>
<p><em>SubStringComparator</em> - This tests if the given substring appears in a specified byte array.
The comparison is case insensitive.
Only EQUAL and NOT_EQUAL comparisons are valid with this comparator</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The general syntax of a comparator is: <code>ComparatorType:ComparatorValue</code></p>
</div>
<div class="paragraph">
<p>The ComparatorType for the various comparators is as follows:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><em>BinaryComparator</em> - binary</p>
</li>
<li>
<p><em>BinaryPrefixComparator</em> - binaryprefix</p>
</li>
<li>
<p><em>RegexStringComparator</em> - regexstring</p>
</li>
<li>
<p><em>SubStringComparator</em> - substring</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The ComparatorValue can be any value.</p>
</div>
<div class="olist arabic">
<div class="title">Example ComparatorValues</div>
<ol class="arabic">
<li>
<p><code>binary:abc</code> will match everything that is lexicographically greater than "abc"</p>
</li>
<li>
<p><code>binaryprefix:abc</code> will match everything whose first 3 characters are lexicographically equal to "abc"</p>
</li>
<li>
<p><code>regexstring:ab*yz</code> will match everything that doesn&#8217;t begin with "ab" and ends with "yz"</p>
</li>
<li>
<p><code>substring:abc123</code> will match everything that begins with the substring "abc123"</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="examplephpclientprogram"><a class="anchor" href="#examplephpclientprogram"></a>107.6. Example PHP Client Program that uses the Filter Language</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="php"><span class="inline-delimiter">&lt;?</span>
<span class="predefined">$_SERVER</span>[<span class="string"><span class="delimiter">'</span><span class="content">PHP_ROOT</span><span class="delimiter">'</span></span>] = <span class="predefined">realpath</span>(<span class="predefined">dirname</span>(<span class="predefined-constant">__FILE__</span>).<span class="string"><span class="delimiter">'</span><span class="content">/..</span><span class="delimiter">'</span></span>);
<span class="predefined">require_once</span> <span class="predefined">$_SERVER</span>[<span class="string"><span class="delimiter">'</span><span class="content">PHP_ROOT</span><span class="delimiter">'</span></span>].<span class="string"><span class="delimiter">'</span><span class="content">/flib/__flib.php</span><span class="delimiter">'</span></span>;
flib_init(<span class="constant">FLIB_CONTEXT_SCRIPT</span>);
require_module(<span class="string"><span class="delimiter">'</span><span class="content">storage/hbase</span><span class="delimiter">'</span></span>);
<span class="local-variable">$hbase</span> = <span class="keyword">new</span> <span class="constant">HBase</span>(<span class="string"><span class="delimiter">'</span><span class="content">&lt;server_name_running_thrift_server&gt;</span><span class="delimiter">'</span></span>, &lt;port on which thrift server is running&gt;);
<span class="local-variable">$hbase</span>-&gt;open();
<span class="local-variable">$client</span> = <span class="local-variable">$hbase</span>-&gt;getClient();
<span class="local-variable">$result</span> = <span class="local-variable">$client</span>-&gt;scannerOpenWithFilterString(<span class="string"><span class="delimiter">'</span><span class="content">table_name</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">&quot;</span><span class="content">(PrefixFilter ('row2') AND (QualifierFilter (&gt;=, 'binary:xyz'))) AND (TimestampsFilter ( 123, 456))</span><span class="delimiter">&quot;</span></span>);
<span class="local-variable">$to_print</span> = <span class="local-variable">$client</span>-&gt;scannerGetList(<span class="local-variable">$result</span>,<span class="integer">1</span>);
<span class="keyword">while</span> (<span class="local-variable">$to_print</span>) {
<span class="predefined">print_r</span>(<span class="local-variable">$to_print</span>);
<span class="local-variable">$to_print</span> = <span class="local-variable">$client</span>-&gt;scannerGetList(<span class="local-variable">$result</span>,<span class="integer">1</span>);
}
<span class="local-variable">$client</span>-&gt;scannerClose(<span class="local-variable">$result</span>);
<span class="inline-delimiter">?&gt;</span></code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_example_filter_strings"><a class="anchor" href="#_example_filter_strings"></a>107.7. Example Filter Strings</h3>
<div class="ulist">
<ul>
<li>
<p><code>"PrefixFilter ('Row') AND PageFilter (1) AND FirstKeyOnlyFilter ()"</code> will return all key-value pairs that match the following conditions:</p>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The row containing the key-value should have prefix <em>Row</em></p>
</li>
<li>
<p>The key-value must be located in the first row of the table</p>
</li>
<li>
<p>The key-value pair must be the first key-value in the row</p>
</li>
</ol>
</div>
</li>
<li>
<p><code>"(RowFilter (=, 'binary:Row 1') AND TimeStampsFilter (74689, 89734)) OR ColumnRangeFilter ('abc', true, 'xyz', false))"</code> will return all key-value pairs that match both the following conditions:</p>
<div class="ulist">
<ul>
<li>
<p>The key-value is in a row having row key <em>Row 1</em></p>
</li>
<li>
<p>The key-value must have a timestamp of either 74689 or 89734.</p>
</li>
<li>
<p>Or it must match the following condition:</p>
<div class="ulist">
<ul>
<li>
<p>The key-value pair must be in a column that is lexicographically &gt;= abc and &lt; xyz </p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</li>
<li>
<p><code>"SKIP ValueFilter (0)"</code> will skip the entire row if any of the values in the row is not 0</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="individualfiltersyntax"><a class="anchor" href="#individualfiltersyntax"></a>107.8. Individual Filter Syntax</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">KeyOnlyFilter</dt>
<dd>
<p>This filter doesn&#8217;t take any arguments.
It returns only the key component of each key-value.</p>
</dd>
<dt class="hdlist1">FirstKeyOnlyFilter</dt>
<dd>
<p>This filter doesn&#8217;t take any arguments.
It returns only the first key-value from each row.</p>
</dd>
<dt class="hdlist1">PrefixFilter</dt>
<dd>
<p>This filter takes one argument – a prefix of a row key.
It returns only those key-values present in a row that starts with the specified row prefix</p>
</dd>
<dt class="hdlist1">ColumnPrefixFilter</dt>
<dd>
<p>This filter takes one argument – a column prefix.
It returns only those key-values present in a column that starts with the specified column prefix.
The column prefix must be of the form: <code>“qualifier”</code>.</p>
</dd>
<dt class="hdlist1">MultipleColumnPrefixFilter</dt>
<dd>
<p>This filter takes a list of column prefixes.
It returns key-values that are present in a column that starts with any of the specified column prefixes.
Each of the column prefixes must be of the form: <code>“qualifier”</code>.</p>
</dd>
<dt class="hdlist1">ColumnCountGetFilter</dt>
<dd>
<p>This filter takes one argument – a limit.
It returns the first limit number of columns in the table.</p>
</dd>
<dt class="hdlist1">PageFilter</dt>
<dd>
<p>This filter takes one argument – a page size.
It returns page size number of rows from the table.</p>
</dd>
<dt class="hdlist1">ColumnPaginationFilter</dt>
<dd>
<p>This filter takes two arguments – a limit and offset.
It returns limit number of columns after offset number of columns.
It does this for all the rows.</p>
</dd>
<dt class="hdlist1">InclusiveStopFilter</dt>
<dd>
<p>This filter takes one argument – a row key on which to stop scanning.
It returns all key-values present in rows up to and including the specified row.</p>
</dd>
<dt class="hdlist1">TimeStampsFilter</dt>
<dd>
<p>This filter takes a list of timestamps.
It returns those key-values whose timestamps matches any of the specified timestamps.</p>
</dd>
<dt class="hdlist1">RowFilter</dt>
<dd>
<p>This filter takes a compare operator and a comparator.
It compares each row key with the comparator using the compare operator and if the comparison returns true, it returns all the key-values in that row.</p>
</dd>
<dt class="hdlist1">Family Filter</dt>
<dd>
<p>This filter takes a compare operator and a comparator.
It compares each column family name with the comparator using the compare operator and if the comparison returns true, it returns all the Cells in that column family.</p>
</dd>
<dt class="hdlist1">QualifierFilter</dt>
<dd>
<p>This filter takes a compare operator and a comparator.
It compares each qualifier name with the comparator using the compare operator and if the comparison returns true, it returns all the key-values in that column.</p>
</dd>
<dt class="hdlist1">ValueFilter</dt>
<dd>
<p>This filter takes a compare operator and a comparator.
It compares each value with the comparator using the compare operator and if the comparison returns true, it returns that key-value.</p>
</dd>
<dt class="hdlist1">DependentColumnFilter</dt>
<dd>
<p>This filter takes two arguments – a family and a qualifier.
It tries to locate this column in each row and returns all key-values in that row that have the same timestamp.
If the row doesn&#8217;t contain the specified column – none of the key-values in that row will be returned.</p>
</dd>
<dt class="hdlist1">SingleColumnValueFilter</dt>
<dd>
<p>This filter takes a column family, a qualifier, a compare operator and a comparator.
If the specified column is not found – all the columns of that row will be emitted.
If the column is found and the comparison with the comparator returns true, all the columns of the row will be emitted.
If the condition fails, the row will not be emitted.</p>
</dd>
<dt class="hdlist1">SingleColumnValueExcludeFilter</dt>
<dd>
<p>This filter takes the same arguments and behaves same as SingleColumnValueFilter – however, if the column is found and the condition passes, all the columns of the row will be emitted except for the tested column value.</p>
</dd>
<dt class="hdlist1">ColumnRangeFilter</dt>
<dd>
<p>This filter is used for selecting only those keys with columns that are between minColumn and maxColumn.
It also takes two boolean variables to indicate whether to include the minColumn and maxColumn or not.</p>
</dd>
</dl>
</div>
</div>
</div>
</div>
<h1 id="spark" class="sect0"><a class="anchor" href="#spark"></a>HBase and Spark</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p><a href="https://spark.apache.org/">Apache Spark</a> is a software framework that is used
to process data in memory in a distributed manner, and is replacing MapReduce in
many use cases.</p>
</div>
<div class="paragraph">
<p>Spark itself is out of scope of this document, please refer to the Spark site for
more information on the Spark project and subprojects. This document will focus
on 4 main interaction points between Spark and HBase. Those interaction points are:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Basic Spark</dt>
<dd>
<p>The ability to have an HBase Connection at any point in your Spark DAG.</p>
</dd>
<dt class="hdlist1">Spark Streaming</dt>
<dd>
<p>The ability to have an HBase Connection at any point in your Spark Streaming
application.</p>
</dd>
<dt class="hdlist1">Spark Bulk Load</dt>
<dd>
<p>The ability to write directly to HBase HFiles for bulk insertion into HBase</p>
</dd>
<dt class="hdlist1">SparkSQL/DataFrames</dt>
<dd>
<p>The ability to write SparkSQL that draws on tables that are represented in HBase.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>The following sections will walk through examples of all these interaction points.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_basic_spark"><a class="anchor" href="#_basic_spark"></a>108. Basic Spark</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This section discusses Spark HBase integration at the lowest and simplest levels.
All the other interaction points are built upon the concepts that will be described
here.</p>
</div>
<div class="paragraph">
<p>At the root of all Spark and HBase integration is the HBaseContext. The HBaseContext
takes in HBase configurations and pushes them to the Spark executors. This allows
us to have an HBase Connection per Spark Executor in a static location.</p>
</div>
<div class="paragraph">
<p>For reference, Spark Executors can be on the same nodes as the Region Servers or
on different nodes, there is no dependence on co-location. Think of every Spark
Executor as a multi-threaded client application. This allows any Spark Tasks
running on the executors to access the shared Connection object.</p>
</div>
<div class="exampleblock">
<div class="title">Example 34. HBaseContext Usage Example</div>
<div class="content">
<div class="paragraph">
<p>This example shows how HBaseContext can be used to do a <code>foreachPartition</code> on a RDD
in Scala:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val sc = new SparkContext(&quot;local&quot;, &quot;test&quot;)
val config = new HBaseConfiguration()
...
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseForeachPartition(hbaseContext, (it, conn) =&gt; {
val bufferedMutator = conn.getBufferedMutator(TableName.valueOf(&quot;t1&quot;))
it.foreach((putRecord) =&gt; {
. val put = new Put(putRecord._1)
. putRecord._2.foreach((putValue) =&gt; put.addColumn(putValue._1, putValue._2, putValue._3))
. bufferedMutator.mutate(put)
})
bufferedMutator.flush()
bufferedMutator.close()
})</code></pre>
</div>
</div>
<div class="paragraph">
<p>Here is the same example implemented in Java:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">JavaSparkContext jsc = <span class="keyword">new</span> JavaSparkContext(sparkConf);
<span class="keyword">try</span> {
<span class="predefined-type">List</span>&lt;<span class="type">byte</span><span class="type">[]</span>&gt; list = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;&gt;();
list.add(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span>));
...
list.add(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">5</span><span class="delimiter">&quot;</span></span>));
JavaRDD&lt;<span class="type">byte</span><span class="type">[]</span>&gt; rdd = jsc.parallelize(list);
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = <span class="keyword">new</span> JavaHBaseContext(jsc, conf);
hbaseContext.foreachPartition(rdd,
<span class="keyword">new</span> VoidFunction&lt;Tuple2&lt;<span class="predefined-type">Iterator</span>&lt;<span class="type">byte</span><span class="type">[]</span>&gt;, <span class="predefined-type">Connection</span>&gt;&gt;() {
<span class="directive">public</span> <span class="type">void</span> call(Tuple2&lt;<span class="predefined-type">Iterator</span>&lt;<span class="type">byte</span><span class="type">[]</span>&gt;, <span class="predefined-type">Connection</span>&gt; t)
<span class="directive">throws</span> <span class="exception">Exception</span> {
Table table = t._2().getTable(TableName.valueOf(tableName));
BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
<span class="keyword">while</span> (t._1().hasNext()) {
<span class="type">byte</span><span class="type">[]</span> b = t._1().next();
<span class="predefined-type">Result</span> r = table.get(<span class="keyword">new</span> Get(b));
<span class="keyword">if</span> (r.getExists()) {
mutator.mutate(<span class="keyword">new</span> Put(b));
}
}
mutator.flush();
mutator.close();
table.close();
}
});
} <span class="keyword">finally</span> {
jsc.stop();
}</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>All functionality between Spark and HBase will be supported both in Scala and in
Java, with the exception of SparkSQL which will support any language that is
supported by Spark. For the remaining of this documentation we will focus on
Scala examples.</p>
</div>
<div class="paragraph">
<p>The examples above illustrate how to do a foreachPartition with a connection. A
number of other Spark base functions are supported out of the box:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>bulkPut</code></dt>
<dd>
<p>For massively parallel sending of puts to HBase</p>
</dd>
<dt class="hdlist1"><code>bulkDelete</code></dt>
<dd>
<p>For massively parallel sending of deletes to HBase</p>
</dd>
<dt class="hdlist1"><code>bulkGet</code></dt>
<dd>
<p>For massively parallel sending of gets to HBase to create a new RDD</p>
</dd>
<dt class="hdlist1"><code>mapPartition</code></dt>
<dd>
<p>To do a Spark Map function with a Connection object to allow full
access to HBase</p>
</dd>
<dt class="hdlist1"><code>hBaseRDD</code></dt>
<dd>
<p>To simplify a distributed scan to create a RDD</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>For examples of all these functionalities, see the
<a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
in the <a href="https://github.com/apache/hbase-connectors">hbase-connectors</a> repository
(the hbase-spark connectors live outside hbase core in a related,
Apache HBase project maintained, associated repo).</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_spark_streaming"><a class="anchor" href="#_spark_streaming"></a>109. Spark Streaming</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="https://spark.apache.org/streaming/">Spark Streaming</a> is a micro batching stream
processing framework built on top of Spark. HBase and Spark Streaming make great
companions in that HBase can help serve the following benefits alongside Spark
Streaming.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A place to grab reference data or profile data on the fly</p>
</li>
<li>
<p>A place to store counts or aggregates in a way that supports Spark Streaming&#8217;s
promise of <em>only once processing</em>.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The <a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
with Spark Streaming is similar to its normal Spark integration points, in that the following
commands are possible straight off a Spark Streaming DStream.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>bulkPut</code></dt>
<dd>
<p>For massively parallel sending of puts to HBase</p>
</dd>
<dt class="hdlist1"><code>bulkDelete</code></dt>
<dd>
<p>For massively parallel sending of deletes to HBase</p>
</dd>
<dt class="hdlist1"><code>bulkGet</code></dt>
<dd>
<p>For massively parallel sending of gets to HBase to create a new RDD</p>
</dd>
<dt class="hdlist1"><code>mapPartition</code></dt>
<dd>
<p>To do a Spark Map function with a Connection object to allow full
access to HBase</p>
</dd>
<dt class="hdlist1"><code>hBaseRDD</code></dt>
<dd>
<p>To simplify a distributed scan to create a RDD</p>
</dd>
</dl>
</div>
<div class="exampleblock">
<div class="title">Example 35. <code>bulkPut</code> Example with DStreams</div>
<div class="content">
<div class="paragraph">
<p>Below is an example of bulkPut with DStreams. It is very close in feel to the RDD
bulk put.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val sc = new SparkContext(&quot;local&quot;, &quot;test&quot;)
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val ssc = new StreamingContext(sc, Milliseconds(200))
val rdd1 = ...
val rdd2 = ...
val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
Array[Byte], Array[Byte])])]]()
queue += rdd1
queue += rdd2
val dStream = ssc.queueStream(queue)
dStream.hbaseBulkPut(
hbaseContext,
TableName.valueOf(tableName),
(putRecord) =&gt; {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) =&gt; put.addColumn(putValue._1, putValue._2, putValue._3))
put
})</code></pre>
</div>
</div>
<div class="paragraph">
<p>There are three inputs to the <code>hbaseBulkPut</code> function.
The hbaseContext that carries the configuration broadcast information link
to the HBase Connections in the executor, the table name of the table we are
putting data into, and a function that will convert a record in the DStream
into an HBase Put object.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_bulk_load"><a class="anchor" href="#_bulk_load"></a>110. Bulk Load</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are two options for bulk loading data into HBase with Spark. There is the
basic bulk load functionality that will work for cases where your rows have
millions of columns and cases where your columns are not consolidated and
partitioned before the map side of the Spark bulk load process.</p>
</div>
<div class="paragraph">
<p>There is also a thin record bulk load option with Spark. This second option is
designed for tables that have less then 10k columns per row. The advantage
of this second option is higher throughput and less over-all load on the Spark
shuffle operation.</p>
</div>
<div class="paragraph">
<p>Both implementations work more or less like the MapReduce bulk load process in
that a partitioner partitions the rowkeys based on region splits and
the row keys are sent to the reducers in order, so that HFiles can be written
out directly from the reduce phase.</p>
</div>
<div class="paragraph">
<p>In Spark terms, the bulk load will be implemented around a Spark
<code>repartitionAndSortWithinPartitions</code> followed by a Spark <code>foreachPartition</code>.</p>
</div>
<div class="paragraph">
<p>First lets look at an example of using the basic bulk load functionality</p>
</div>
<div class="exampleblock">
<div class="title">Example 36. Bulk Loading Example</div>
<div class="content">
<div class="paragraph">
<p>The following example shows bulk loading with Spark.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val sc = new SparkContext(&quot;local&quot;, &quot;test&quot;)
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes(&quot;1&quot;),
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;a&quot;), Bytes.toBytes(&quot;foo1&quot;))),
(Bytes.toBytes(&quot;3&quot;),
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;b&quot;), Bytes.toBytes(&quot;foo2.b&quot;))), ...
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t =&gt; {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>The <code>hbaseBulkLoad</code> function takes three required parameters:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The table name of the table we intend to bulk load too</p>
</li>
<li>
<p>A function that will convert a record in the RDD to a tuple key value par. With
the tuple key being a KeyFamilyQualifer object and the value being the cell value.
The KeyFamilyQualifer object will hold the RowKey, Column Family, and Column Qualifier.
The shuffle will partition on the RowKey but will sort by all three values.</p>
</li>
<li>
<p>The temporary path for the HFile to be written out too</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Following the Spark bulk load command, use the HBase&#8217;s LoadIncrementalHFiles object
to load the newly created HFiles into HBase.</p>
</div>
<div class="paragraph">
<div class="title">Additional Parameters for Bulk Loading with Spark</div>
<p>You can set the following attributes with additional parameter options on hbaseBulkLoad.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Max file size of the HFiles</p>
</li>
<li>
<p>A flag to exclude HFiles from compactions</p>
</li>
<li>
<p>Column Family settings for compression, bloomType, blockSize, and dataBlockEncoding</p>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 37. Using Additional Parameters</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val sc = new SparkContext(&quot;local&quot;, &quot;test&quot;)
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes(&quot;1&quot;),
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;a&quot;), Bytes.toBytes(&quot;foo1&quot;))),
(Bytes.toBytes(&quot;3&quot;),
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;b&quot;), Bytes.toBytes(&quot;foo2.b&quot;))), ...
val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
val f1Options = new FamilyHFileWriteOptions(&quot;GZ&quot;, &quot;ROW&quot;, 128, &quot;PREFIX&quot;)
familyHBaseWriterOptions.put(Bytes.toBytes(&quot;columnFamily1&quot;), f1Options)
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t =&gt; {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath,
familyHBaseWriterOptions,
compactionExclude = false,
HConstants.DEFAULT_MAX_FILE_SIZE)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>Now lets look at how you would call the thin record bulk load implementation</p>
</div>
<div class="exampleblock">
<div class="title">Example 38. Using thin record bulk load</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val sc = new SparkContext(&quot;local&quot;, &quot;test&quot;)
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(&quot;1&quot;,
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;a&quot;), Bytes.toBytes(&quot;foo1&quot;))),
(&quot;3&quot;,
(Bytes.toBytes(columnFamily1), Bytes.toBytes(&quot;b&quot;), Bytes.toBytes(&quot;foo2.b&quot;))), ...
rdd.hbaseBulkLoadThinRows(hbaseContext,
TableName.valueOf(tableName),
t =&gt; {
val rowKey = t._1
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f =&gt; {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
},
stagingFolder.getPath,
new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
compactionExclude = false,
20)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))</code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>Note that the big difference in using bulk load for thin rows is the function
returns a tuple with the first value being the row key and the second value
being an object of FamiliesQualifiersValues, which will contain all the
values for this row for all column families.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_sparksql_dataframes"><a class="anchor" href="#_sparksql_dataframes"></a>111. SparkSQL/DataFrames</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The <a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
leverages
<a href="https://databricks.com/blog/2015/01/09/spark-sql-data-sources-api-unified-data-access-for-the-spark-platform.html">DataSource API</a>
(<a href="https://issues.apache.org/jira/browse/SPARK-3247">SPARK-3247</a>)
introduced in Spark-1.2.0, which bridges the gap between simple HBase KV store and complex
relational SQL queries and enables users to perform complex data analytical work
on top of HBase using Spark. HBase Dataframe is a standard Spark Dataframe, and is able to
interact with any other data sources such as Hive, Orc, Parquet, JSON, etc.
The <a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
applies critical techniques such as partition pruning, column pruning,
predicate pushdown and data locality.</p>
</div>
<div class="paragraph">
<p>To use the
<a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
connector, users need to define the Catalog for the schema mapping
between HBase and Spark tables, prepare the data and populate the HBase table,
then load the HBase DataFrame. After that, users can do integrated query and access records
in HBase tables with SQL query. The following illustrates the basic procedure.</p>
</div>
<div class="sect2">
<h3 id="_define_catalog"><a class="anchor" href="#_define_catalog"></a>111.1. Define catalog</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">def catalog = s&quot;&quot;&quot;{
       |&quot;table&quot;:{&quot;namespace&quot;:&quot;default&quot;, &quot;name&quot;:&quot;table1&quot;},
       |&quot;rowkey&quot;:&quot;key&quot;,
       |&quot;columns&quot;:{
         |&quot;col0&quot;:{&quot;cf&quot;:&quot;rowkey&quot;, &quot;col&quot;:&quot;key&quot;, &quot;type&quot;:&quot;string&quot;},
         |&quot;col1&quot;:{&quot;cf&quot;:&quot;cf1&quot;, &quot;col&quot;:&quot;col1&quot;, &quot;type&quot;:&quot;boolean&quot;},
         |&quot;col2&quot;:{&quot;cf&quot;:&quot;cf2&quot;, &quot;col&quot;:&quot;col2&quot;, &quot;type&quot;:&quot;double&quot;},
         |&quot;col3&quot;:{&quot;cf&quot;:&quot;cf3&quot;, &quot;col&quot;:&quot;col3&quot;, &quot;type&quot;:&quot;float&quot;},
         |&quot;col4&quot;:{&quot;cf&quot;:&quot;cf4&quot;, &quot;col&quot;:&quot;col4&quot;, &quot;type&quot;:&quot;int&quot;},
         |&quot;col5&quot;:{&quot;cf&quot;:&quot;cf5&quot;, &quot;col&quot;:&quot;col5&quot;, &quot;type&quot;:&quot;bigint&quot;},
         |&quot;col6&quot;:{&quot;cf&quot;:&quot;cf6&quot;, &quot;col&quot;:&quot;col6&quot;, &quot;type&quot;:&quot;smallint&quot;},
         |&quot;col7&quot;:{&quot;cf&quot;:&quot;cf7&quot;, &quot;col&quot;:&quot;col7&quot;, &quot;type&quot;:&quot;string&quot;},
         |&quot;col8&quot;:{&quot;cf&quot;:&quot;cf8&quot;, &quot;col&quot;:&quot;col8&quot;, &quot;type&quot;:&quot;tinyint&quot;}
       |}
     |}&quot;&quot;&quot;.stripMargin</code></pre>
</div>
</div>
<div class="paragraph">
<p>Catalog defines a mapping between HBase and Spark tables. There are two critical parts of this catalog.
One is the rowkey definition and the other is the mapping between table column in Spark and
the column family and column qualifier in HBase. The above defines a schema for a HBase table
with name as table1, row key as key and a number of columns (col1 <code>-</code> col8). Note that the rowkey
also has to be defined in details as a column (col0), which has a specific cf (rowkey).</p>
</div>
</div>
<div class="sect2">
<h3 id="_save_the_dataframe"><a class="anchor" href="#_save_the_dataframe"></a>111.2. Save the DataFrame</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">case class HBaseRecord(
col0: String,
col1: Boolean,
col2: Double,
col3: Float,
col4: Int,       
col5: Long,
col6: Short,
col7: String,
col8: Byte)
object HBaseRecord
{                                                                                                             
def apply(i: Int, t: String): HBaseRecord = {
val s = s&quot;&quot;&quot;row${&quot;%03d&quot;.format(i)}&quot;&quot;&quot;       
HBaseRecord(s,
i % 2 == 0,
i.toDouble,
i.toFloat,  
i,
i.toLong,
i.toShort,  
s&quot;String$i: $t&quot;,      
i.toByte)
}
}
val data = (0 to 255).map { i =&gt;  HBaseRecord(i, &quot;extra&quot;)}
sc.parallelize(data).toDF.write.options(
 Map(HBaseTableCatalog.tableCatalog -&gt; catalog, HBaseTableCatalog.newTable -&gt; &quot;5&quot;))
 .format(&quot;org.apache.hadoop.hbase.spark &quot;)
 .save()</code></pre>
</div>
</div>
<div class="paragraph">
<p><code>data</code> prepared by the user is a local Scala collection which has 256 HBaseRecord objects.
<code>sc.parallelize(data)</code> function distributes <code>data</code> to form an RDD. <code>toDF</code> returns a DataFrame.
<code>write</code> function returns a DataFrameWriter used to write the DataFrame to external storage
systems (e.g. HBase here). Given a DataFrame with specified schema <code>catalog</code>, <code>save</code> function
will create an HBase table with 5 regions and save the DataFrame inside.</p>
</div>
</div>
<div class="sect2">
<h3 id="_load_the_dataframe"><a class="anchor" href="#_load_the_dataframe"></a>111.3. Load the DataFrame</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog-&gt;cat))
.format(&quot;org.apache.hadoop.hbase.spark&quot;)
.load()
}
val df = withCatalog(catalog)</code></pre>
</div>
</div>
<div class="paragraph">
<p>In ‘withCatalog’ function, sqlContext is a variable of SQLContext, which is the entry point
for working with structured data (rows and columns) in Spark.
<code>read</code> returns a DataFrameReader that can be used to read data in as a DataFrame.
<code>option</code> function adds input options for the underlying data source to the DataFrameReader,
and <code>format</code> function specifies the input data source format for the DataFrameReader.
The <code>load()</code> function loads input in as a DataFrame. The date frame <code>df</code> returned
by <code>withCatalog</code> function could be used to access HBase table, such as 4.4 and 4.5.</p>
</div>
</div>
<div class="sect2">
<h3 id="_language_integrated_query"><a class="anchor" href="#_language_integrated_query"></a>111.4. Language Integrated Query</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val s = df.filter(($&quot;col0&quot; &lt;= &quot;row050&quot; &amp;&amp; $&quot;col0&quot; &gt; &quot;row040&quot;) ||
$&quot;col0&quot; === &quot;row005&quot; ||
$&quot;col0&quot; &lt;= &quot;row005&quot;)
.select(&quot;col0&quot;, &quot;col1&quot;, &quot;col4&quot;)
s.show</code></pre>
</div>
</div>
<div class="paragraph">
<p>DataFrame can do various operations, such as join, sort, select, filter, orderBy and so on.
<code>df.filter</code> above filters rows using the given SQL expression. <code>select</code> selects a set of columns:
<code>col0</code>, <code>col1</code> and <code>col4</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_sql_query"><a class="anchor" href="#_sql_query"></a>111.5. SQL Query</h3>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">df.registerTempTable(&quot;table1&quot;)
sqlContext.sql(&quot;select count(col1) from table1&quot;).show</code></pre>
</div>
</div>
<div class="paragraph">
<p><code>registerTempTable</code> registers <code>df</code> DataFrame as a temporary table using the table name <code>table1</code>.
The lifetime of this temporary table is tied to the SQLContext that was used to create <code>df</code>.
<code>sqlContext.sql</code> function allows the user to execute SQL queries.</p>
</div>
</div>
<div class="sect2">
<h3 id="_others"><a class="anchor" href="#_others"></a>111.6. Others</h3>
<div class="exampleblock">
<div class="title">Example 39. Query with different timestamps</div>
<div class="content">
<div class="paragraph">
<p>In HBaseSparkConf, four parameters related to timestamp can be set. They are TIMESTAMP,
MIN_TIMESTAMP, MAX_TIMESTAMP and MAX_VERSIONS respectively. Users can query records with
different timestamps or time ranges with MIN_TIMESTAMP and MAX_TIMESTAMP. In the meantime,
use concrete value instead of tsSpecified and oldMs in the examples below.</p>
</div>
<div class="paragraph">
<p>The example below shows how to load df DataFrame with different timestamps.
tsSpecified is specified by the user.
HBaseTableCatalog defines the HBase and Relation relation schema.
writeCatalog defines catalog for the schema mapping.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -&gt; writeCatalog, HBaseSparkConf.TIMESTAMP -&gt; tsSpecified.toString))
.format(&quot;org.apache.hadoop.hbase.spark&quot;)
.load()</code></pre>
</div>
</div>
<div class="paragraph">
<p>The example below shows how to load df DataFrame with different time ranges.
oldMs is specified by the user.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -&gt; writeCatalog, HBaseSparkConf.MIN_TIMESTAMP -&gt; &quot;0&quot;,
HBaseSparkConf.MAX_TIMESTAMP -&gt; oldMs.toString))
.format(&quot;org.apache.hadoop.hbase.spark&quot;)
.load()</code></pre>
</div>
</div>
<div class="paragraph">
<p>After loading df DataFrame, users can query data.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">df.registerTempTable(&quot;table&quot;)
sqlContext.sql(&quot;select count(col1) from table&quot;).show</code></pre>
</div>
</div>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 40. Native Avro support</div>
<div class="content">
<div class="paragraph">
<p>The <a href="https://github.com/apache/hbase-connectors/tree/master/spark">hbase-spark integration</a>
connector supports different data formats like Avro, JSON, etc. The use case below
shows how spark supports Avro. Users can persist the Avro record into HBase directly. Internally,
the Avro schema is converted to a native Spark Catalyst data type automatically.
Note that both key-value parts in an HBase table can be defined in Avro format.</p>
</div>
<div class="paragraph">
<p>1) Define catalog for the schema mapping:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">def catalog = s&quot;&quot;&quot;{
|&quot;table&quot;:{&quot;namespace&quot;:&quot;default&quot;, &quot;name&quot;:&quot;Avrotable&quot;},
|&quot;rowkey&quot;:&quot;key&quot;,
|&quot;columns&quot;:{
|&quot;col0&quot;:{&quot;cf&quot;:&quot;rowkey&quot;, &quot;col&quot;:&quot;key&quot;, &quot;type&quot;:&quot;string&quot;},
|&quot;col1&quot;:{&quot;cf&quot;:&quot;cf1&quot;, &quot;col&quot;:&quot;col1&quot;, &quot;type&quot;:&quot;binary&quot;}
|}
|}&quot;&quot;&quot;.stripMargin</code></pre>
</div>
</div>
<div class="paragraph">
<p><code>catalog</code> is a schema for a HBase table named <code>Avrotable</code>. row key as key and
one column col1. The rowkey also has to be defined in details as a column (col0),
which has a specific cf (rowkey).</p>
</div>
<div class="paragraph">
<p>2) Prepare the Data:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala"> object AvroHBaseRecord {
val schemaString =
s&quot;&quot;&quot;{&quot;namespace&quot;: &quot;example.avro&quot;,
| &quot;type&quot;: &quot;record&quot;, &quot;name&quot;: &quot;User&quot;,
| &quot;fields&quot;: [
| {&quot;name&quot;: &quot;name&quot;, &quot;type&quot;: &quot;string&quot;},
| {&quot;name&quot;: &quot;favorite_number&quot;, &quot;type&quot;: [&quot;int&quot;, &quot;null&quot;]},
| {&quot;name&quot;: &quot;favorite_color&quot;, &quot;type&quot;: [&quot;string&quot;, &quot;null&quot;]},
| {&quot;name&quot;: &quot;favorite_array&quot;, &quot;type&quot;: {&quot;type&quot;: &quot;array&quot;, &quot;items&quot;: &quot;string&quot;}},
| {&quot;name&quot;: &quot;favorite_map&quot;, &quot;type&quot;: {&quot;type&quot;: &quot;map&quot;, &quot;values&quot;: &quot;int&quot;}}
| ] }&quot;&quot;&quot;.stripMargin
val avroSchema: Schema = {
val p = new Schema.Parser
p.parse(schemaString)
}
def apply(i: Int): AvroHBaseRecord = {
val user = new GenericData.Record(avroSchema);
user.put(&quot;name&quot;, s&quot;name${&quot;%03d&quot;.format(i)}&quot;)
user.put(&quot;favorite_number&quot;, i)
user.put(&quot;favorite_color&quot;, s&quot;color${&quot;%03d&quot;.format(i)}&quot;)
val favoriteArray = new GenericData.Array[String](2, avroSchema.getField(&quot;favorite_array&quot;).schema())
favoriteArray.add(s&quot;number${i}&quot;)
favoriteArray.add(s&quot;number${i+1}&quot;)
user.put(&quot;favorite_array&quot;, favoriteArray)
import collection.JavaConverters._
val favoriteMap = Map[String, Int]((&quot;key1&quot; -&gt; i), (&quot;key2&quot; -&gt; (i+1))).asJava
user.put(&quot;favorite_map&quot;, favoriteMap)
val avroByte = AvroSedes.serialize(user, avroSchema)
AvroHBaseRecord(s&quot;name${&quot;%03d&quot;.format(i)}&quot;, avroByte)
}
}
val data = (0 to 255).map { i =&gt;
AvroHBaseRecord(i)
}</code></pre>
</div>
</div>
<div class="paragraph">
<p><code>schemaString</code> is defined first, then it is parsed to get <code>avroSchema</code>. <code>avroSchema</code> is used to
generate <code>AvroHBaseRecord</code>. <code>data</code> prepared by users is a local Scala collection
which has 256 <code>AvroHBaseRecord</code> objects.</p>
</div>
<div class="paragraph">
<p>3) Save DataFrame:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala"> sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -&gt; catalog, HBaseTableCatalog.newTable -&gt; &quot;5&quot;))
.format(&quot;org.apache.spark.sql.execution.datasources.hbase&quot;)
.save()</code></pre>
</div>
</div>
<div class="paragraph">
<p>Given a data frame with specified schema <code>catalog</code>, above will create an HBase table with 5
regions and save the data frame inside.</p>
</div>
<div class="paragraph">
<p>4) Load the DataFrame</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala">def avroCatalog = s&quot;&quot;&quot;{
|&quot;table&quot;:{&quot;namespace&quot;:&quot;default&quot;, &quot;name&quot;:&quot;avrotable&quot;},
|&quot;rowkey&quot;:&quot;key&quot;,
|&quot;columns&quot;:{
|&quot;col0&quot;:{&quot;cf&quot;:&quot;rowkey&quot;, &quot;col&quot;:&quot;key&quot;, &quot;type&quot;:&quot;string&quot;},
|&quot;col1&quot;:{&quot;cf&quot;:&quot;cf1&quot;, &quot;col&quot;:&quot;col1&quot;, &quot;avro&quot;:&quot;avroSchema&quot;}
|}
|}&quot;&quot;&quot;.stripMargin
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(&quot;avroSchema&quot; -&gt; AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -&gt; avroCatalog))
.format(&quot;org.apache.spark.sql.execution.datasources.hbase&quot;)
.load()
}
val df = withCatalog(catalog)</code></pre>
</div>
</div>
<div class="paragraph">
<p>In <code>withCatalog</code> function, <code>read</code> returns a DataFrameReader that can be used to read data in as a DataFrame.
The <code>option</code> function adds input options for the underlying data source to the DataFrameReader.
There are two options: one is to set <code>avroSchema</code> as <code>AvroHBaseRecord.schemaString</code>, and one is to
set <code>HBaseTableCatalog.tableCatalog</code> as <code>avroCatalog</code>. The <code>load()</code> function loads input in as a DataFrame.
The date frame <code>df</code> returned by <code>withCatalog</code> function could be used to access the HBase table.</p>
</div>
<div class="paragraph">
<p>5) SQL Query</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="scala"> df.registerTempTable(&quot;avrotable&quot;)
val c = sqlContext.sql(&quot;select count(1) from avrotable&quot;).</code></pre>
</div>
</div>
<div class="paragraph">
<p>After loading df DataFrame, users can query data. registerTempTable registers df DataFrame
as a temporary table using the table name avrotable. <code>sqlContext.sql</code> function allows the
user to execute SQL queries.</p>
</div>
</div>
</div>
</div>
</div>
</div>
<h1 id="cp" class="sect0"><a class="anchor" href="#cp"></a>Apache HBase Coprocessors</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>HBase Coprocessors are modeled after Google BigTable&#8217;s coprocessor implementation
(<a href="http://research.google.com/people/jeff/SOCC2010-keynote-slides.pdf" class="bare">http://research.google.com/people/jeff/SOCC2010-keynote-slides.pdf</a> pages 41-42.).</p>
</div>
<div class="paragraph">
<p>The coprocessor framework provides mechanisms for running your custom code directly on
the RegionServers managing your data. Efforts are ongoing to bridge gaps between HBase&#8217;s
implementation and BigTable&#8217;s architecture. For more information see
<a href="https://issues.apache.org/jira/browse/HBASE-4047">HBASE-4047</a>.</p>
</div>
<div class="paragraph">
<p>The information in this chapter is primarily sourced and heavily reused from the following
resources:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Mingjie Lai&#8217;s blog post
<a href="https://blogs.apache.org/hbase/entry/coprocessor_introduction">Coprocessor Introduction</a>.</p>
</li>
<li>
<p>Gaurav Bhardwaj&#8217;s blog post
<a href="http://www.3pillarglobal.com/insights/hbase-coprocessors">The How To Of HBase Coprocessors</a>.</p>
</li>
</ol>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Use Coprocessors At Your Own Risk</div>
<div class="paragraph">
<p>Coprocessors are an advanced feature of HBase and are intended to be used by system
developers only. Because coprocessor code runs directly on the RegionServer and has
direct access to your data, they introduce the risk of data corruption, man-in-the-middle
attacks, or other malicious data access. Currently, there is no mechanism to prevent
data corruption by coprocessors, though work is underway on
<a href="https://issues.apache.org/jira/browse/HBASE-4047">HBASE-4047</a>.</p>
</div>
<div class="paragraph">
<p>+
In addition, there is no resource isolation, so a well-intentioned but misbehaving
coprocessor can severely degrade cluster performance and stability.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_coprocessor_overview"><a class="anchor" href="#_coprocessor_overview"></a>112. Coprocessor Overview</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In HBase, you fetch data using a <code>Get</code> or <code>Scan</code>, whereas in an RDBMS you use a SQL
query. In order to fetch only the relevant data, you filter it using a HBase
<a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html">Filter</a>
, whereas in an RDBMS you use a <code>WHERE</code> predicate.</p>
</div>
<div class="paragraph">
<p>After fetching the data, you perform computations on it. This paradigm works well
for "small data" with a few thousand rows and several columns. However, when you scale
to billions of rows and millions of columns, moving large amounts of data across your
network will create bottlenecks at the network layer, and the client needs to be powerful
enough and have enough memory to handle the large amounts of data and the computations.
In addition, the client code can grow large and complex.</p>
</div>
<div class="paragraph">
<p>In this scenario, coprocessors might make sense. You can put the business computation
code into a coprocessor which runs on the RegionServer, in the same location as the
data, and returns the result to the client.</p>
</div>
<div class="paragraph">
<p>This is only one scenario where using coprocessors can provide benefit. Following
are some analogies which may help to explain some of the benefits of coprocessors.</p>
</div>
<div class="sect2">
<h3 id="cp_analogies"><a class="anchor" href="#cp_analogies"></a>112.1. Coprocessor Analogies</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">Triggers and Stored Procedure</dt>
<dd>
<p>An Observer coprocessor is similar to a trigger in a RDBMS in that it executes
your code either before or after a specific event (such as a <code>Get</code> or <code>Put</code>)
occurs. An endpoint coprocessor is similar to a stored procedure in a RDBMS
because it allows you to perform custom computations on the data on the
RegionServer itself, rather than on the client.</p>
</dd>
<dt class="hdlist1">MapReduce</dt>
<dd>
<p>MapReduce operates on the principle of moving the computation to the location of
the data. Coprocessors operate on the same principal.</p>
</dd>
<dt class="hdlist1">AOP</dt>
<dd>
<p>If you are familiar with Aspect Oriented Programming (AOP), you can think of a coprocessor
as applying advice by intercepting a request and then running some custom code,
before passing the request on to its final destination (or even changing the destination).</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_coprocessor_implementation_overview"><a class="anchor" href="#_coprocessor_implementation_overview"></a>112.2. Coprocessor Implementation Overview</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Your class should implement one of the Coprocessor interfaces -
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/Coprocessor.html">Coprocessor</a>,
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html">RegionObserver</a>,
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/CoprocessorService.html">CoprocessorService</a> - to name a few.</p>
</li>
<li>
<p>Load the coprocessor, either statically (from the configuration) or dynamically,
using HBase Shell. For more details see <a href="#cp_loading">Loading Coprocessors</a>.</p>
</li>
<li>
<p>Call the coprocessor from your client-side code. HBase handles the coprocessor
transparently.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The framework API is provided in the
<a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html">coprocessor</a>
package.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_types_of_coprocessors"><a class="anchor" href="#_types_of_coprocessors"></a>113. Types of Coprocessors</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_observer_coprocessors"><a class="anchor" href="#_observer_coprocessors"></a>113.1. Observer Coprocessors</h3>
<div class="paragraph">
<p>Observer coprocessors are triggered either before or after a specific event occurs.
Observers that happen before an event use methods that start with a <code>pre</code> prefix,
such as <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#prePut-org.apache.hadoop.hbase.coprocessor.ObserverContext-org.apache.hadoop.hbase.client.Put-org.apache.hadoop.hbase.wal.WALEdit-org.apache.hadoop.hbase.client.Durability-"><code>prePut</code></a>. Observers that happen just after an event override methods that start
with a <code>post</code> prefix, such as <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#postPut-org.apache.hadoop.hbase.coprocessor.ObserverContext-org.apache.hadoop.hbase.client.Put-org.apache.hadoop.hbase.wal.WALEdit-org.apache.hadoop.hbase.client.Durability-"><code>postPut</code></a>.</p>
</div>
<div class="sect3">
<h4 id="_use_cases_for_observer_coprocessors"><a class="anchor" href="#_use_cases_for_observer_coprocessors"></a>113.1.1. Use Cases for Observer Coprocessors</h4>
<div class="dlist">
<dl>
<dt class="hdlist1">Security</dt>
<dd>
<p>Before performing a <code>Get</code> or <code>Put</code> operation, you can check for permission using
<code>preGet</code> or <code>prePut</code> methods.</p>
</dd>
<dt class="hdlist1">Referential Integrity</dt>
<dd>
<p>HBase does not directly support the RDBMS concept of refential integrity, also known
as foreign keys. You can use a coprocessor to enforce such integrity. For instance,
if you have a business rule that every insert to the <code>users</code> table must be followed
by a corresponding entry in the <code>user_daily_attendance</code> table, you could implement
a coprocessor to use the <code>prePut</code> method on <code>user</code> to insert a record into <code>user_daily_attendance</code>.</p>
</dd>
<dt class="hdlist1">Secondary Indexes</dt>
<dd>
<p>You can use a coprocessor to maintain secondary indexes. For more information, see
<a href="https://wiki.apache.org/hadoop/Hbase/SecondaryIndexing">SecondaryIndexing</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="_types_of_observer_coprocessor"><a class="anchor" href="#_types_of_observer_coprocessor"></a>113.1.2. Types of Observer Coprocessor</h4>
<div class="dlist">
<dl>
<dt class="hdlist1">RegionObserver</dt>
<dd>
<p>A RegionObserver coprocessor allows you to observe events on a region, such as <code>Get</code>
and <code>Put</code> operations. See
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html">RegionObserver</a>.</p>
</dd>
<dt class="hdlist1">RegionServerObserver</dt>
<dd>
<p>A RegionServerObserver allows you to observe events related to the RegionServer&#8217;s
operation, such as starting, stopping, or performing merges, commits, or rollbacks.
See
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.html">RegionServerObserver</a>.</p>
</dd>
<dt class="hdlist1">MasterObserver</dt>
<dd>
<p>A MasterObserver allows you to observe events related to the HBase Master, such
as table creation, deletion, or schema modification. See
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html">MasterObserver</a>.</p>
</dd>
<dt class="hdlist1">WalObserver</dt>
<dd>
<p>A WalObserver allows you to observe events related to writes to the Write-Ahead
Log (WAL). See
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/WALObserver.html">WALObserver</a>.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p><a href="#cp_example">Examples</a> provides working examples of observer coprocessors.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="cpeps"><a class="anchor" href="#cpeps"></a>113.2. Endpoint Coprocessor</h3>
<div class="paragraph">
<p>Endpoint processors allow you to perform computation at the location of the data.
See <a href="#cp_analogies">Coprocessor Analogy</a>. An example is the need to calculate a running
average or summation for an entire table which spans hundreds of regions.</p>
</div>
<div class="paragraph">
<p>In contrast to observer coprocessors, where your code is run transparently, endpoint
coprocessors must be explicitly invoked using the
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html#coprocessorService-java.lang.Class-byte:A-byte:A-org.apache.hadoop.hbase.client.coprocessor.Batch.Call-">CoprocessorService()</a>
method available in
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html">Table</a>
or
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</a>.</p>
</div>
<div class="paragraph">
<p>Starting with HBase 0.96, endpoint coprocessors are implemented using Google Protocol
Buffers (protobuf). For more details on protobuf, see Google&#8217;s
<a href="https://developers.google.com/protocol-buffers/docs/proto">Protocol Buffer Guide</a>.
Endpoints Coprocessor written in version 0.94 are not compatible with version 0.96 or later.
See
<a href="https://issues.apache.org/jira/browse/HBASE-5448">HBASE-5448</a>). To upgrade your
HBase cluster from 0.94 or earlier to 0.96 or later, you need to reimplement your
coprocessor.</p>
</div>
<div class="paragraph">
<p>Coprocessor Endpoints should make no use of HBase internals and
only avail of public APIs; ideally a CPEP should depend on Interfaces
and data structures only. This is not always possible but beware
that doing so makes the Endpoint brittle, liable to breakage as HBase
internals evolve. HBase internal APIs annotated as private or evolving
do not have to respect semantic versioning rules or general java rules on
deprecation before removal. While generated protobuf files are
absent the hbase audience annotations&#8201;&#8212;&#8201;they are created by the
protobuf protoc tool which knows nothing of how HBase works&#8201;&#8212;&#8201;they should be consided <code>@InterfaceAudience.Private</code> so are liable to
change.</p>
</div>
<div class="paragraph">
<p><a href="#cp_example">Examples</a> provides working examples of endpoint coprocessors.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="cp_loading"><a class="anchor" href="#cp_loading"></a>114. Loading Coprocessors</h2>
<div class="sectionbody">
<div class="paragraph">
<p>To make your coprocessor available to HBase, it must be <em>loaded</em>, either statically
(through the HBase configuration) or dynamically (using HBase Shell or the Java API).</p>
</div>
<div class="sect2">
<h3 id="_static_loading"><a class="anchor" href="#_static_loading"></a>114.1. Static Loading</h3>
<div class="paragraph">
<p>Follow these steps to statically load your coprocessor. Keep in mind that you must
restart HBase to unload a coprocessor that has been loaded statically.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Define the Coprocessor in <em>hbase-site.xml</em>, with a &lt;property&gt; element with a &lt;name&gt;
and a &lt;value&gt; sub-element. The &lt;name&gt; should be one of the following:</p>
<div class="ulist">
<ul>
<li>
<p><code>hbase.coprocessor.region.classes</code> for RegionObservers and Endpoints.</p>
</li>
<li>
<p><code>hbase.coprocessor.wal.classes</code> for WALObservers.</p>
</li>
<li>
<p><code>hbase.coprocessor.master.classes</code> for MasterObservers.</p>
<div class="paragraph">
<p>&lt;value&gt; must contain the fully-qualified class name of your coprocessor&#8217;s implementation
class.</p>
</div>
<div class="paragraph">
<p>For example to load a Coprocessor (implemented in class SumEndPoint.java) you have to create
following entry in RegionServer&#8217;s 'hbase-site.xml' file (generally located under 'conf' directory):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.myname.hbase.coprocessor.endpoint.SumEndPoint<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>If multiple classes are specified for loading, the class names must be comma-separated.
The framework attempts to load all the configured classes using the default class loader.
Therefore, the jar file must reside on the server-side HBase classpath.</p>
</div>
<div class="paragraph">
<p>Coprocessors which are loaded in this way will be active on all regions of all tables.
These are also called system Coprocessor.
The first listed Coprocessors will be assigned the priority <code>Coprocessor.Priority.SYSTEM</code>.
Each subsequent coprocessor in the list will have its priority value incremented by one (which
reduces its priority, because priorities have the natural sort order of Integers).</p>
</div>
<div class="paragraph">
<p>When calling out to registered observers, the framework executes their callbacks methods in the
sorted order of their priority.<br>
Ties are broken arbitrarily.</p>
</div>
</li>
</ul>
</div>
</li>
<li>
<p>Put your code on HBase&#8217;s classpath. One easy way to do this is to drop the jar
(containing you code and all the dependencies) into the <code>lib/</code> directory in the
HBase installation.</p>
</li>
<li>
<p>Restart HBase.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_static_unloading"><a class="anchor" href="#_static_unloading"></a>114.2. Static Unloading</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Delete the coprocessor&#8217;s &lt;property&gt; element, including sub-elements, from <code>hbase-site.xml</code>.</p>
</li>
<li>
<p>Restart HBase.</p>
</li>
<li>
<p>Optionally, remove the coprocessor&#8217;s JAR file from the classpath or HBase&#8217;s <code>lib/</code>
directory.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_dynamic_loading"><a class="anchor" href="#_dynamic_loading"></a>114.3. Dynamic Loading</h3>
<div class="paragraph">
<p>You can also load a coprocessor dynamically, without restarting HBase. This may seem
preferable to static loading, but dynamically loaded coprocessors are loaded on a
per-table basis, and are only available to the table for which they were loaded. For
this reason, dynamically loaded tables are sometimes called <strong>Table Coprocessor</strong>.</p>
</div>
<div class="paragraph">
<p>In addition, dynamically loading a coprocessor acts as a schema change on the table,
and the table must be taken offline to load the coprocessor.</p>
</div>
<div class="paragraph">
<p>There are three ways to dynamically load Coprocessor.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Assumptions</div>
<div class="paragraph">
<p>The below mentioned instructions makes the following assumptions:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A JAR called <code>coprocessor.jar</code> contains the Coprocessor implementation along with all of its
dependencies.</p>
</li>
<li>
<p>The JAR is available in HDFS in some location like
<code>hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/coprocessor.jar</code>.</p>
</li>
</ul>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="load_coprocessor_in_shell"><a class="anchor" href="#load_coprocessor_in_shell"></a>114.3.1. Using HBase Shell</h4>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Disable the table using HBase Shell:</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; disable <span class="string"><span class="delimiter">'</span><span class="content">users</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</li>
<li>
<p>Load the Coprocessor, using a command like the following:</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase alter <span class="string"><span class="delimiter">'</span><span class="content">users</span><span class="delimiter">'</span></span>, METHOD =&gt; <span class="string"><span class="delimiter">'</span><span class="content">table_att</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">Coprocessor</span><span class="delimiter">'</span></span>=&gt;<span class="string"><span class="delimiter">'</span><span class="content">hdfs://&lt;namenode&gt;:&lt;port&gt;/
user/&lt;hadoop-user&gt;/coprocessor.jar| org.myname.hbase.Coprocessor.RegionObserverExample|1073741823|
arg1=1,arg2=2</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The Coprocessor framework will try to read the class information from the coprocessor table
attribute value.
The value contains four pieces of information which are separated by the pipe (<code>|</code>) character.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>File path: The jar file containing the Coprocessor implementation must be in a location where
all region servers can read it.<br>
You could copy the file onto the local disk on each region server, but it is recommended to store
it in HDFS.<br>
<a href="https://issues.apache.org/jira/browse/HBASE-14548">HBASE-14548</a> allows a directory containing the jars
or some wildcards to be specified, such as: hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/ or
hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/*.jar. Please note that if a directory is specified,
all jar files(.jar) in the directory are added. It does not search for files in sub-directories.
Do not use a wildcard if you would like to specify a directory. This enhancement applies to the
usage via the JAVA API as well.</p>
</li>
<li>
<p>Class name: The full class name of the Coprocessor.</p>
</li>
<li>
<p>Priority: An integer. The framework will determine the execution sequence of all configured
observers registered at the same hook using priorities. This field can be left blank. In that
case the framework will assign a default priority value.</p>
</li>
<li>
<p>Arguments (Optional): This field is passed to the Coprocessor implementation. This is optional.</p>
</li>
</ul>
</div>
</li>
<li>
<p>Enable the table.</p>
<div class="listingblock">
<div class="content">
<pre>hbase(main):003:0&gt; enable 'users'</pre>
</div>
</div>
</li>
<li>
<p>Verify that the coprocessor loaded:</p>
<div class="listingblock">
<div class="content">
<pre>hbase(main):04:0&gt; describe 'users'</pre>
</div>
</div>
<div class="paragraph">
<p>The coprocessor should be listed in the <code>TABLE_ATTRIBUTES</code>.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_using_the_java_api_all_hbase_versions"><a class="anchor" href="#_using_the_java_api_all_hbase_versions"></a>114.3.2. Using the Java API (all HBase versions)</h4>
<div class="paragraph">
<p>The following Java code shows how to use the <code>setValue()</code> method of <code>HTableDescriptor</code>
to load a coprocessor on the <code>users</code> table.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">TableName tableName = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">users</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">String</span> path = <span class="string"><span class="delimiter">&quot;</span><span class="content">hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/coprocessor.jar</span><span class="delimiter">&quot;</span></span>;
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = <span class="keyword">new</span> HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">personalDet</span><span class="delimiter">&quot;</span></span>);
columnFamily1.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily1);
HColumnDescriptor columnFamily2 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">salaryDet</span><span class="delimiter">&quot;</span></span>);
columnFamily2.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily2);
hTableDescriptor.setValue(<span class="string"><span class="delimiter">&quot;</span><span class="content">COPROCESSOR$1</span><span class="delimiter">&quot;</span></span>, path + <span class="string"><span class="delimiter">&quot;</span><span class="content">|</span><span class="delimiter">&quot;</span></span>
+ RegionObserverExample.class.getCanonicalName() + <span class="string"><span class="delimiter">&quot;</span><span class="content">|</span><span class="delimiter">&quot;</span></span>
+ Coprocessor.PRIORITY_USER);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_using_the_java_api_hbase_0_96_only"><a class="anchor" href="#_using_the_java_api_hbase_0_96_only"></a>114.3.3. Using the Java API (HBase 0.96+ only)</h4>
<div class="paragraph">
<p>In HBase 0.96 and newer, the <code>addCoprocessor()</code> method of <code>HTableDescriptor</code> provides
an easier way to load a coprocessor dynamically.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">TableName tableName = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">users</span><span class="delimiter">&quot;</span></span>);
Path path = <span class="keyword">new</span> Path(<span class="string"><span class="delimiter">&quot;</span><span class="content">hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/coprocessor.jar</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = <span class="keyword">new</span> HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">personalDet</span><span class="delimiter">&quot;</span></span>);
columnFamily1.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily1);
HColumnDescriptor columnFamily2 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">salaryDet</span><span class="delimiter">&quot;</span></span>);
columnFamily2.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily2);
hTableDescriptor.addCoprocessor(RegionObserverExample.class.getCanonicalName(), path,
Coprocessor.PRIORITY_USER, <span class="predefined-constant">null</span>);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);</code></pre>
</div>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
There is no guarantee that the framework will load a given Coprocessor successfully.
For example, the shell command neither guarantees a jar file exists at a particular location nor
verifies whether the given class is actually contained in the jar file.
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_dynamic_unloading"><a class="anchor" href="#_dynamic_unloading"></a>114.4. Dynamic Unloading</h3>
<div class="sect3">
<h4 id="_using_hbase_shell"><a class="anchor" href="#_using_hbase_shell"></a>114.4.1. Using HBase Shell</h4>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Disable the table.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; disable <span class="string"><span class="delimiter">'</span><span class="content">users</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</li>
<li>
<p>Alter the table to remove the coprocessor.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; alter <span class="string"><span class="delimiter">'</span><span class="content">users</span><span class="delimiter">'</span></span>, METHOD =&gt; <span class="string"><span class="delimiter">'</span><span class="content">table_att_unset</span><span class="delimiter">'</span></span>, NAME =&gt; <span class="string"><span class="delimiter">'</span><span class="content">coprocessor$1</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</li>
<li>
<p>Enable the table.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase&gt; enable <span class="string"><span class="delimiter">'</span><span class="content">users</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_using_the_java_api"><a class="anchor" href="#_using_the_java_api"></a>114.4.2. Using the Java API</h4>
<div class="paragraph">
<p>Reload the table definition without setting the value of the coprocessor either by
using <code>setValue()</code> or <code>addCoprocessor()</code> methods. This will remove any coprocessor
attached to the table.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">TableName tableName = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">users</span><span class="delimiter">&quot;</span></span>);
<span class="predefined-type">String</span> path = <span class="string"><span class="delimiter">&quot;</span><span class="content">hdfs://&lt;namenode&gt;:&lt;port&gt;/user/&lt;hadoop-user&gt;/coprocessor.jar</span><span class="delimiter">&quot;</span></span>;
<span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
admin.disableTable(tableName);
HTableDescriptor hTableDescriptor = <span class="keyword">new</span> HTableDescriptor(tableName);
HColumnDescriptor columnFamily1 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">personalDet</span><span class="delimiter">&quot;</span></span>);
columnFamily1.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily1);
HColumnDescriptor columnFamily2 = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">salaryDet</span><span class="delimiter">&quot;</span></span>);
columnFamily2.setMaxVersions(<span class="integer">3</span>);
hTableDescriptor.addFamily(columnFamily2);
admin.modifyTable(tableName, hTableDescriptor);
admin.enableTable(tableName);</code></pre>
</div>
</div>
<div class="paragraph">
<p>In HBase 0.96 and newer, you can instead use the <code>removeCoprocessor()</code> method of the
<code>HTableDescriptor</code> class.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="cp_example"><a class="anchor" href="#cp_example"></a>115. Examples</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase ships examples for Observer Coprocessor.</p>
</div>
<div class="paragraph">
<p>A more detailed example is given below.</p>
</div>
<div class="paragraph">
<p>These examples assume a table called <code>users</code>, which has two column families <code>personalDet</code>
and <code>salaryDet</code>, containing personal and salary details. Below is the graphical representation
of the <code>users</code> table.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 21. Users Table</caption>
<colgroup>
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2857%;">
<col style="width: 14.2858%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top"></th>
<th class="tableblock halign-left valign-top" colspan="3">personalDet</th>
<th class="tableblock halign-left valign-top" colspan="3">salaryDet</th>
</tr>
</thead>
<tfoot>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">jverne</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Jules</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Verne</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">02/08/1828</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">12000</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">9000</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">3000</p></td>
</tr>
</tfoot>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>rowkey</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>name</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>lastname</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>dob</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>gross</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>net</strong></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>allowances</strong></p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">admin</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Admin</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Admin</p></td>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top" colspan="3"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">cdickens</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Charles</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Dickens</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">02/07/1812</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">10000</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">8000</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">2000</p></td>
</tr>
</tbody>
</table>
<div class="sect2">
<h3 id="_observer_example"><a class="anchor" href="#_observer_example"></a>115.1. Observer Example</h3>
<div class="paragraph">
<p>The following Observer coprocessor prevents the details of the user <code>admin</code> from being
returned in a <code>Get</code> or <code>Scan</code> of the <code>users</code> table.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Write a class that implements the
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionCoprocessor.html">RegionCoprocessor</a>,
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html">RegionObserver</a>
class.</p>
</li>
<li>
<p>Override the <code>preGetOp()</code> method (the <code>preGet()</code> method is deprecated) to check
whether the client has queried for the rowkey with value <code>admin</code>. If so, return an
empty result. Otherwise, process the request as normal.</p>
</li>
<li>
<p>Put your code and dependencies in a JAR file.</p>
</li>
<li>
<p>Place the JAR in HDFS where HBase can locate it.</p>
</li>
<li>
<p>Load the Coprocessor.</p>
</li>
<li>
<p>Write a simple program to test it.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Following are the implementation of the above steps:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">RegionObserverExample</span> <span class="directive">implements</span> RegionCoprocessor, RegionObserver {
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ADMIN = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">admin</span><span class="delimiter">&quot;</span></span>);
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> COLUMN_FAMILY = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">details</span><span class="delimiter">&quot;</span></span>);
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> COLUMN = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">Admin_det</span><span class="delimiter">&quot;</span></span>);
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> VALUE = Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">You can't see Admin details</span><span class="delimiter">&quot;</span></span>);
<span class="annotation">@Override</span>
<span class="directive">public</span> Optional&lt;RegionObserver&gt; getRegionObserver() {
<span class="keyword">return</span> Optional.of(<span class="local-variable">this</span>);
}
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="type">void</span> preGetOp(<span class="directive">final</span> ObserverContext&lt;RegionCoprocessorEnvironment&gt; e, <span class="directive">final</span> Get get, <span class="directive">final</span> <span class="predefined-type">List</span>&lt;Cell&gt; results)
<span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">if</span> (Bytes.equals(get.getRow(),ADMIN)) {
Cell c = CellUtil.createCell(get.getRow(),COLUMN_FAMILY, COLUMN,
<span class="predefined-type">System</span>.currentTimeMillis(), (<span class="type">byte</span>)<span class="integer">4</span>, VALUE);
results.add(c);
e.bypass();
}
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Overriding the <code>preGetOp()</code> will only work for <code>Get</code> operations. You also need to override
the <code>preScannerOpen()</code> method to filter the <code>admin</code> row from scan results.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="annotation">@Override</span>
<span class="directive">public</span> RegionScanner preScannerOpen(<span class="directive">final</span> ObserverContext&lt;RegionCoprocessorEnvironment&gt; e, <span class="directive">final</span> Scan scan,
<span class="directive">final</span> RegionScanner s) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="predefined-type">Filter</span> filter = <span class="keyword">new</span> RowFilter(CompareOp.NOT_EQUAL, <span class="keyword">new</span> BinaryComparator(ADMIN));
scan.setFilter(filter);
<span class="keyword">return</span> s;
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>This method works but there is a <em>side effect</em>. If the client has used a filter in
its scan, that filter will be replaced by this filter. Instead, you can explicitly
remove any <code>admin</code> results from the scan:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="annotation">@Override</span>
<span class="directive">public</span> <span class="type">boolean</span> postScannerNext(<span class="directive">final</span> ObserverContext&lt;RegionCoprocessorEnvironment&gt; e, <span class="directive">final</span> InternalScanner s,
<span class="directive">final</span> <span class="predefined-type">List</span>&lt;<span class="predefined-type">Result</span>&gt; results, <span class="directive">final</span> <span class="type">int</span> limit, <span class="directive">final</span> <span class="type">boolean</span> hasMore) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="predefined-type">Result</span> result = <span class="predefined-constant">null</span>;
<span class="predefined-type">Iterator</span>&lt;<span class="predefined-type">Result</span>&gt; iterator = results.iterator();
<span class="keyword">while</span> (iterator.hasNext()) {
result = iterator.next();
<span class="keyword">if</span> (Bytes.equals(result.getRow(), ROWKEY)) {
iterator.remove();
<span class="keyword">break</span>;
}
}
<span class="keyword">return</span> hasMore;
}</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_endpoint_example"><a class="anchor" href="#_endpoint_example"></a>115.2. Endpoint Example</h3>
<div class="paragraph">
<p>Still using the <code>users</code> table, this example implements a coprocessor to calculate
the sum of all employee salaries, using an endpoint coprocessor.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Create a '.proto' file defining your service.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">option java_package = <span class="string"><span class="delimiter">&quot;</span><span class="content">org.myname.hbase.coprocessor.autogenerated</span><span class="delimiter">&quot;</span></span>;
option java_outer_classname = <span class="string"><span class="delimiter">&quot;</span><span class="content">Sum</span><span class="delimiter">&quot;</span></span>;
option java_generic_services = <span class="predefined-constant">true</span>;
option java_generate_equals_and_hash = <span class="predefined-constant">true</span>;
option optimize_for = SPEED;
message SumRequest {
required string family = <span class="integer">1</span>;
required string column = <span class="integer">2</span>;
}
message SumResponse {
required int64 sum = <span class="integer">1</span> [<span class="keyword">default</span> = <span class="integer">0</span>];
}
service SumService {
rpc getSum(SumRequest)
returns (SumResponse);
}</code></pre>
</div>
</div>
</li>
<li>
<p>Execute the <code>protoc</code> command to generate the Java code from the above .proto' file.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> mkdir src
<span class="error">$</span> protoc --java_out=src ./sum.proto</code></pre>
</div>
</div>
<div class="paragraph">
<p>This will generate a class call <code>Sum.java</code>.</p>
</div>
</li>
<li>
<p>Write a class that extends the generated service class, implement the <code>Coprocessor</code>
and <code>CoprocessorService</code> classes, and override the service method.</p>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
If you load a coprocessor from <code>hbase-site.xml</code> and then load the same coprocessor
again using HBase Shell, it will be loaded a second time. The same class will
exist twice, and the second instance will have a higher ID (and thus a lower priority).
The effect is that the duplicate coprocessor is effectively ignored.
</td>
</tr>
</table>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">SumEndPoint</span> <span class="directive">extends</span> Sum.SumService <span class="directive">implements</span> Coprocessor, CoprocessorService {
<span class="directive">private</span> RegionCoprocessorEnvironment env;
<span class="annotation">@Override</span>
<span class="directive">public</span> Service getService() {
<span class="keyword">return</span> <span class="local-variable">this</span>;
}
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="type">void</span> start(CoprocessorEnvironment env) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="keyword">if</span> (env <span class="keyword">instanceof</span> RegionCoprocessorEnvironment) {
<span class="local-variable">this</span>.env = (RegionCoprocessorEnvironment)env;
} <span class="keyword">else</span> {
<span class="keyword">throw</span> <span class="keyword">new</span> CoprocessorException(<span class="string"><span class="delimiter">&quot;</span><span class="content">Must be loaded on a table region!</span><span class="delimiter">&quot;</span></span>);
}
}
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="type">void</span> stop(CoprocessorEnvironment env) <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="comment">// do nothing</span>
}
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="type">void</span> getSum(RpcController controller, Sum.SumRequest request, RpcCallback&lt;Sum.SumResponse&gt; done) {
Scan scan = <span class="keyword">new</span> Scan();
scan.addFamily(Bytes.toBytes(request.getFamily()));
scan.addColumn(Bytes.toBytes(request.getFamily()), Bytes.toBytes(request.getColumn()));
Sum.SumResponse response = <span class="predefined-constant">null</span>;
InternalScanner scanner = <span class="predefined-constant">null</span>;
<span class="keyword">try</span> {
scanner = env.getRegion().getScanner(scan);
<span class="predefined-type">List</span>&lt;Cell&gt; results = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;&gt;();
<span class="type">boolean</span> hasMore = <span class="predefined-constant">false</span>;
<span class="type">long</span> sum = <span class="integer">0L</span>;
<span class="keyword">do</span> {
hasMore = scanner.next(results);
<span class="keyword">for</span> (Cell cell : results) {
sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
}
results.clear();
} <span class="keyword">while</span> (hasMore);
response = Sum.SumResponse.newBuilder().setSum(sum).build();
} <span class="keyword">catch</span> (<span class="exception">IOException</span> ioe) {
ResponseConverter.setControllerException(controller, ioe);
} <span class="keyword">finally</span> {
<span class="keyword">if</span> (scanner != <span class="predefined-constant">null</span>) {
<span class="keyword">try</span> {
scanner.close();
} <span class="keyword">catch</span> (<span class="exception">IOException</span> ignored) {}
}
}
done.run(response);
}
}</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
TableName tableName = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">users</span><span class="delimiter">&quot;</span></span>);
Table table = connection.getTable(tableName);
<span class="directive">final</span> Sum.SumRequest request = Sum.SumRequest.newBuilder().setFamily(<span class="string"><span class="delimiter">&quot;</span><span class="content">salaryDet</span><span class="delimiter">&quot;</span></span>).setColumn(<span class="string"><span class="delimiter">&quot;</span><span class="content">gross</span><span class="delimiter">&quot;</span></span>).build();
<span class="keyword">try</span> {
<span class="predefined-type">Map</span>&lt;<span class="type">byte</span><span class="type">[]</span>, <span class="predefined-type">Long</span>&gt; results = table.coprocessorService(
Sum.SumService.class,
<span class="predefined-constant">null</span>, <span class="comment">/* start key */</span>
<span class="predefined-constant">null</span>, <span class="comment">/* end key */</span>
<span class="keyword">new</span> Batch.Call&lt;Sum.SumService, <span class="predefined-type">Long</span>&gt;() {
<span class="annotation">@Override</span>
<span class="directive">public</span> <span class="predefined-type">Long</span> call(Sum.SumService aggregate) <span class="directive">throws</span> <span class="exception">IOException</span> {
BlockingRpcCallback&lt;Sum.SumResponse&gt; rpcCallback = <span class="keyword">new</span> BlockingRpcCallback&lt;&gt;();
aggregate.getSum(<span class="predefined-constant">null</span>, request, rpcCallback);
Sum.SumResponse response = rpcCallback.get();
<span class="keyword">return</span> response.hasSum() ? response.getSum() : <span class="integer">0L</span>;
}
}
);
<span class="keyword">for</span> (<span class="predefined-type">Long</span> sum : results.values()) {
<span class="predefined-type">System</span>.out.println(<span class="string"><span class="delimiter">&quot;</span><span class="content">Sum = </span><span class="delimiter">&quot;</span></span> + sum);
}
} <span class="keyword">catch</span> (ServiceException e) {
e.printStackTrace();
} <span class="keyword">catch</span> (<span class="predefined-type">Throwable</span> e) {
e.printStackTrace();
}</code></pre>
</div>
</div>
</li>
<li>
<p>Load the Coprocessor.</p>
</li>
<li>
<p>Write a client code to call the Coprocessor.</p>
</li>
</ol>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_guidelines_for_deploying_a_coprocessor"><a class="anchor" href="#_guidelines_for_deploying_a_coprocessor"></a>116. Guidelines For Deploying A Coprocessor</h2>
<div class="sectionbody">
<div class="dlist">
<dl>
<dt class="hdlist1">Bundling Coprocessors</dt>
<dd>
<p>You can bundle all classes for a coprocessor into a
single JAR on the RegionServer&#8217;s classpath, for easy deployment. Otherwise,
place all dependencies on the RegionServer&#8217;s classpath so that they can be
loaded during RegionServer start-up. The classpath for a RegionServer is set
in the RegionServer&#8217;s <code>hbase-env.sh</code> file.</p>
</dd>
<dt class="hdlist1">Automating Deployment</dt>
<dd>
<p>You can use a tool such as Puppet, Chef, or
Ansible to ship the JAR for the coprocessor to the required location on your
RegionServers' filesystems and restart each RegionServer, to automate
coprocessor deployment. Details for such set-ups are out of scope of this
document.</p>
</dd>
<dt class="hdlist1">Updating a Coprocessor</dt>
<dd>
<p>Deploying a new version of a given coprocessor is not as simple as disabling it,
replacing the JAR, and re-enabling the coprocessor. This is because you cannot
reload a class in a JVM unless you delete all the current references to it.
Since the current JVM has reference to the existing coprocessor, you must restart
the JVM, by restarting the RegionServer, in order to replace it. This behavior
is not expected to change.</p>
</dd>
<dt class="hdlist1">Coprocessor Logging</dt>
<dd>
<p>The Coprocessor framework does not provide an API for logging beyond standard Java
logging.</p>
</dd>
<dt class="hdlist1">Coprocessor Configuration</dt>
<dd>
<p>If you do not want to load coprocessors from the HBase Shell, you can add their configuration
properties to <code>hbase-site.xml</code>. In <a href="#load_coprocessor_in_shell">Using HBase Shell</a>, two arguments are
set: <code>arg1=1,arg2=2</code>. These could have been added to <code>hbase-site.xml</code> as follows:</p>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>arg1<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>1<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>arg2<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>2<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Then you can read the configuration using code like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> conf = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(conf);
TableName tableName = TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">users</span><span class="delimiter">&quot;</span></span>);
Table table = connection.getTable(tableName);
Get get = <span class="keyword">new</span> Get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">admin</span><span class="delimiter">&quot;</span></span>));
<span class="predefined-type">Result</span> result = table.get(get);
<span class="keyword">for</span> (Cell c : result.rawCells()) {
<span class="predefined-type">System</span>.out.println(Bytes.toString(CellUtil.cloneRow(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content">==&gt; </span><span class="delimiter">&quot;</span></span> + Bytes.toString(CellUtil.cloneFamily(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content">{</span><span class="delimiter">&quot;</span></span> + Bytes.toString(CellUtil.cloneQualifier(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content">:</span><span class="delimiter">&quot;</span></span> + Bytes.toLong(CellUtil.cloneValue(c)) + <span class="string"><span class="delimiter">&quot;</span><span class="content">}</span><span class="delimiter">&quot;</span></span>);
}
Scan scan = <span class="keyword">new</span> Scan();
ResultScanner scanner = table.getScanner(scan);
<span class="keyword">for</span> (<span class="predefined-type">Result</span> res : scanner) {
<span class="keyword">for</span> (Cell c : res.rawCells()) {
<span class="predefined-type">System</span>.out.println(Bytes.toString(CellUtil.cloneRow(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content"> ==&gt; </span><span class="delimiter">&quot;</span></span> + Bytes.toString(CellUtil.cloneFamily(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content"> {</span><span class="delimiter">&quot;</span></span> + Bytes.toString(CellUtil.cloneQualifier(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content">:</span><span class="delimiter">&quot;</span></span> + Bytes.toLong(CellUtil.cloneValue(c))
+ <span class="string"><span class="delimiter">&quot;</span><span class="content">}</span><span class="delimiter">&quot;</span></span>);
}
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_restricting_coprocessor_usage"><a class="anchor" href="#_restricting_coprocessor_usage"></a>117. Restricting Coprocessor Usage</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Restricting arbitrary user coprocessors can be a big concern in multitenant environments. HBase provides a continuum of options for ensuring only expected coprocessors are running:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.coprocessor.enabled</code>: Enables or disables all coprocessors. This will limit the functionality of HBase, as disabling all coprocessors will disable some security providers. An example coproccessor so affected is <code>org.apache.hadoop.hbase.security.access.AccessController</code>.</p>
<div class="ulist">
<ul>
<li>
<p><code>hbase.coprocessor.user.enabled</code>: Enables or disables loading coprocessors on tables (i.e. user coprocessors).</p>
</li>
<li>
<p>One can statically load coprocessors via the following tunables in <code>hbase-site.xml</code>:</p>
<div class="ulist">
<ul>
<li>
<p><code>hbase.coprocessor.regionserver.classes</code>: A comma-separated list of coprocessors that are loaded by region servers</p>
</li>
<li>
<p><code>hbase.coprocessor.region.classes</code>: A comma-separated list of RegionObserver and Endpoint coprocessors</p>
</li>
<li>
<p><code>hbase.coprocessor.user.region.classes</code>: A comma-separated list of coprocessors that are loaded by all regions</p>
</li>
<li>
<p><code>hbase.coprocessor.master.classes</code>: A comma-separated list of coprocessors that are loaded by the master (MasterObserver coprocessors)</p>
</li>
<li>
<p><code>hbase.coprocessor.wal.classes</code>: A comma-separated list of WALObserver coprocessors to load</p>
</li>
</ul>
</div>
</li>
<li>
<p><code>hbase.coprocessor.abortonerror</code>: Whether to abort the daemon which has loaded the coprocessor if the coprocessor should error other than <code>IOError</code>. If this is set to false and an access controller coprocessor should have a fatal error the coprocessor will be circumvented, as such in secure installations this is advised to be <code>true</code>; however, one may override this on a per-table basis for user coprocessors, to ensure they do not abort their running region server and are instead unloaded on error.</p>
</li>
<li>
<p><code>hbase.coprocessor.region.whitelist.paths</code>: A comma separated list available for those loading <code>org.apache.hadoop.hbase.security.access.CoprocessorWhitelistMasterObserver</code> whereby one can use the following options to white-list paths from which coprocessors may be loaded.</p>
<div class="ulist">
<ul>
<li>
<p>Coprocessors on the classpath are implicitly white-listed</p>
</li>
<li>
<p><code>*</code> to wildcard all coprocessor paths</p>
</li>
<li>
<p>An entire filesystem (e.g. <code>hdfs://my-cluster/</code>)</p>
</li>
<li>
<p>A wildcard path to be evaluated by <a href="https://commons.apache.org/proper/commons-io/javadocs/api-release/org/apache/commons/io/FilenameUtils.html">FilenameUtils.wildcardMatch</a></p>
</li>
<li>
<p>Note: Path can specify scheme or not (e.g. <code><a href="file:///usr/hbase/lib/coprocessors" class="bare">file:///usr/hbase/lib/coprocessors</a></code> or for all filesystems <code>/usr/hbase/lib/coprocessors</code>)</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
</div>
<h1 id="performance" class="sect0"><a class="anchor" href="#performance"></a>Apache HBase Performance Tuning</h1>
<div class="sect1">
<h2 id="perf.os"><a class="anchor" href="#perf.os"></a>118. Operating System</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="perf.os.ram"><a class="anchor" href="#perf.os.ram"></a>118.1. Memory</h3>
<div class="paragraph">
<p>RAM, RAM, RAM.
Don&#8217;t starve HBase.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.os.64"><a class="anchor" href="#perf.os.64"></a>118.2. 64-bit</h3>
<div class="paragraph">
<p>Use a 64-bit platform (and 64-bit JVM).</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.os.swap"><a class="anchor" href="#perf.os.swap"></a>118.3. Swapping</h3>
<div class="paragraph">
<p>Watch out for swapping.
Set <code>swappiness</code> to 0.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.os.cpu"><a class="anchor" href="#perf.os.cpu"></a>118.4. CPU</h3>
<div class="paragraph">
<p>Make sure you have set up your Hadoop to use native, hardware checksumming.
See link:[hadoop.native.lib].</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.network"><a class="anchor" href="#perf.network"></a>119. Network</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Perhaps the most important factor in avoiding network issues degrading Hadoop and HBase performance is the switching hardware that is used, decisions made early in the scope of the project can cause major problems when you double or triple the size of your cluster (or more).</p>
</div>
<div class="paragraph">
<p>Important items to consider:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Switching capacity of the device</p>
</li>
<li>
<p>Number of systems connected</p>
</li>
<li>
<p>Uplink capacity</p>
</li>
</ul>
</div>
<div class="sect2">
<h3 id="perf.network.1switch"><a class="anchor" href="#perf.network.1switch"></a>119.1. Single Switch</h3>
<div class="paragraph">
<p>The single most important factor in this configuration is that the switching capacity of the hardware is capable of handling the traffic which can be generated by all systems connected to the switch.
Some lower priced commodity hardware can have a slower switching capacity than could be utilized by a full switch.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.network.2switch"><a class="anchor" href="#perf.network.2switch"></a>119.2. Multiple Switches</h3>
<div class="paragraph">
<p>Multiple switches are a potential pitfall in the architecture.
The most common configuration of lower priced hardware is a simple 1Gbps uplink from one switch to another.
This often overlooked pinch point can easily become a bottleneck for cluster communication.
Especially with MapReduce jobs that are both reading and writing a lot of data the communication across this uplink could be saturated.</p>
</div>
<div class="paragraph">
<p>Mitigation of this issue is fairly simple and can be accomplished in multiple ways:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Use appropriate hardware for the scale of the cluster which you&#8217;re attempting to build.</p>
</li>
<li>
<p>Use larger single switch configurations i.e.
single 48 port as opposed to 2x 24 port</p>
</li>
<li>
<p>Configure port trunking for uplinks to utilize multiple interfaces to increase cross switch bandwidth.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="perf.network.multirack"><a class="anchor" href="#perf.network.multirack"></a>119.3. Multiple Racks</h3>
<div class="paragraph">
<p>Multiple rack configurations carry the same potential issues as multiple switches, and can suffer performance degradation from two main areas:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Poor switch capacity performance</p>
</li>
<li>
<p>Insufficient uplink to another rack</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>If the switches in your rack have appropriate switching capacity to handle all the hosts at full speed, the next most likely issue will be caused by homing more of your cluster across racks.
The easiest way to avoid issues when spanning multiple racks is to use port trunking to create a bonded uplink to other racks.
The downside of this method however, is in the overhead of ports that could potentially be used.
An example of this is, creating an 8Gbps port channel from rack A to rack B, using 8 of your 24 ports to communicate between racks gives you a poor ROI, using too few however can mean you&#8217;re not getting the most out of your cluster.</p>
</div>
<div class="paragraph">
<p>Using 10Gbe links between racks will greatly increase performance, and assuming your switches support a 10Gbe uplink or allow for an expansion card will allow you to save your ports for machines as opposed to uplinks.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.network.ints"><a class="anchor" href="#perf.network.ints"></a>119.4. Network Interfaces</h3>
<div class="paragraph">
<p>Are all the network interfaces functioning correctly? Are you sure? See the Troubleshooting Case Study in <a href="#casestudies.slownode">Case Study #1 (Performance Issue On A Single Node)</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.network.call_me_maybe"><a class="anchor" href="#perf.network.call_me_maybe"></a>119.5. Network Consistency and Partition Tolerance</h3>
<div class="paragraph">
<p>The <a href="http://en.wikipedia.org/wiki/CAP_theorem">CAP Theorem</a> states that a distributed system can maintain two out of the following three characteristics:
- *C*onsistency&#8201;&#8212;&#8201;all nodes see the same data.
- *A*vailability&#8201;&#8212;&#8201;every request receives a response about whether it succeeded or failed.
- *P*artition tolerance&#8201;&#8212;&#8201;the system continues to operate even if some of its components become unavailable to the others.</p>
</div>
<div class="paragraph">
<p>HBase favors consistency and partition tolerance, where a decision has to be made. Coda Hale explains why partition tolerance is so important, in <a href="http://codahale.com/you-cant-sacrifice-partition-tolerance/" class="bare">http://codahale.com/you-cant-sacrifice-partition-tolerance/</a>.</p>
</div>
<div class="paragraph">
<p>Robert Yokota used an automated testing framework called <a href="https://aphyr.com/tags/jepsen">Jepson</a> to test HBase&#8217;s partition tolerance in the face of network partitions, using techniques modeled after Aphyr&#8217;s <a href="https://aphyr.com/posts/281-call-me-maybe-carly-rae-jepsen-and-the-perils-of-network-partitions">Call Me Maybe</a> series. The results, available as a <a href="https://rayokota.wordpress.com/2015/09/30/call-me-maybe-hbase/">blog post</a> and an <a href="https://rayokota.wordpress.com/2015/09/30/call-me-maybe-hbase-addendum/">addendum</a>, show that HBase performs correctly.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="jvm"><a class="anchor" href="#jvm"></a>120. Java</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="gc"><a class="anchor" href="#gc"></a>120.1. The Garbage Collector and Apache HBase</h3>
<div class="sect3">
<h4 id="gcpause"><a class="anchor" href="#gcpause"></a>120.1.1. Long GC pauses</h4>
<div class="paragraph">
<p>In his presentation, <a href="http://www.slideshare.net/cloudera/hbase-hug-presentation">Avoiding Full GCs with MemStore-Local Allocation Buffers</a>, Todd Lipcon describes two cases of stop-the-world garbage collections common in HBase, especially during loading; CMS failure modes and old generation heap fragmentation brought.</p>
</div>
<div class="paragraph">
<p>To address the first, start the CMS earlier than default by adding <code>-XX:CMSInitiatingOccupancyFraction</code> and setting it down from defaults.
Start at 60 or 70 percent (The lower you bring down the threshold, the more GCing is done, the more CPU used). To address the second fragmentation issue, Todd added an experimental facility,
(MSLAB), that must be explicitly enabled in Apache HBase 0.90.x (It&#8217;s defaulted to be <em>on</em> in Apache 0.92.x HBase). Set <code>hbase.hregion.memstore.mslab.enabled</code> to true in your <code>Configuration</code>.
See the cited slides for background and detail.
The latest JVMs do better regards fragmentation so make sure you are running a recent release.
Read down in the message, <a href="http://osdir.com/ml/hotspot-gc-use/2011-11/msg00002.html">Identifying concurrent mode failures caused by fragmentation</a>.
Be aware that when enabled, each MemStore instance will occupy at least an MSLAB instance of memory.
If you have thousands of regions or lots of regions each with many column families, this allocation of MSLAB may be responsible for a good portion of your heap allocation and in an extreme case cause you to OOME.
Disable MSLAB in this case, or lower the amount of memory it uses or float less regions per server.</p>
</div>
<div class="paragraph">
<p>If you have a write-heavy workload, check out <a href="https://issues.apache.org/jira/browse/HBASE-8163">HBASE-8163 MemStoreChunkPool: An improvement for JAVA GC when using MSLAB</a>.
It describes configurations to lower the amount of young GC during write-heavy loadings.
If you do not have HBASE-8163 installed, and you are trying to improve your young GC times, one trick to consider&#8201;&#8212;&#8201;courtesy of our Liang Xie&#8201;&#8212;&#8201;is to set the GC config <code>-XX:PretenureSizeThreshold</code> in <em>hbase-env.sh</em> to be just smaller than the size of <code>hbase.hregion.memstore.mslab.chunksize</code> so MSLAB allocations happen in the tenured space directly rather than first in the young gen.
You&#8217;d do this because these MSLAB allocations are going to likely make it to the old gen anyways and rather than pay the price of a copies between s0 and s1 in eden space followed by the copy up from young to old gen after the MSLABs have achieved sufficient tenure, save a bit of YGC churn and allocate in the old gen directly.</p>
</div>
<div class="paragraph">
<p>Other sources of long GCs can be the JVM itself logging.
See <a href="https://engineering.linkedin.com/blog/2016/02/eliminating-large-jvm-gc-pauses-caused-by-background-io-traffic">Eliminating Large JVM GC Pauses Caused by Background IO Traffic</a></p>
</div>
<div class="paragraph">
<p>For more information about GC logs, see <a href="#trouble.log.gc">JVM Garbage Collection Logs</a>.</p>
</div>
<div class="paragraph">
<p>Consider also enabling the off-heap Block Cache.
This has been shown to mitigate GC pause times.
See <a href="#block.cache">Block Cache</a></p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.configurations"><a class="anchor" href="#perf.configurations"></a>121. HBase Configurations</h2>
<div class="sectionbody">
<div class="paragraph">
<p>See <a href="#recommended_configurations">Recommended Configurations</a>.</p>
</div>
<div class="sect2">
<h3 id="perf.99th.percentile"><a class="anchor" href="#perf.99th.percentile"></a>121.1. Improving the 99th Percentile</h3>
<div class="paragraph">
<p>Try link:[hedged_reads].</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.compactions.and.splits"><a class="anchor" href="#perf.compactions.and.splits"></a>121.2. Managing Compactions</h3>
<div class="paragraph">
<p>For larger systems, managing link:[compactions and splits] may be something you want to consider.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.handlers"><a class="anchor" href="#perf.handlers"></a>121.3. <code>hbase.regionserver.handler.count</code></h3>
<div class="paragraph">
<p>See <a href="#hbase.regionserver.handler.count">[hbase.regionserver.handler.count]</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hfile.block.cache.size"><a class="anchor" href="#perf.hfile.block.cache.size"></a>121.4. <code>hfile.block.cache.size</code></h3>
<div class="paragraph">
<p>See <a href="#hfile.block.cache.size">[hfile.block.cache.size]</a>.
A memory setting for the RegionServer process.</p>
</div>
</div>
<div class="sect2">
<h3 id="blockcache.prefetch"><a class="anchor" href="#blockcache.prefetch"></a>121.5. Prefetch Option for Blockcache</h3>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-9857">HBASE-9857</a> adds a new option to prefetch HFile contents when opening the BlockCache, if a Column family or RegionServer property is set.
This option is available for HBase 0.98.3 and later.
The purpose is to warm the BlockCache as rapidly as possible after the cache is opened, using in-memory table data, and not counting the prefetching as cache misses.
This is great for fast reads, but is not a good idea if the data to be preloaded will not fit into the BlockCache.
It is useful for tuning the IO impact of prefetching versus the time before all data blocks are in cache.</p>
</div>
<div class="paragraph">
<p>To enable prefetching on a given column family, you can use HBase Shell or use the API.</p>
</div>
<div class="listingblock">
<div class="title">Enable Prefetch Using HBase Shell</div>
<div class="content">
<pre>hbase&gt; create 'MyTable', { NAME =&gt; 'myCF', PREFETCH_BLOCKS_ON_OPEN =&gt; 'true' }</pre>
</div>
</div>
<div class="exampleblock">
<div class="title">Example 41. Enable Prefetch Using the API</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">// ...</span>
HTableDescriptor tableDesc = <span class="keyword">new</span> HTableDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">myTable</span><span class="delimiter">&quot;</span></span>);
HColumnDescriptor cfDesc = <span class="keyword">new</span> HColumnDescriptor(<span class="string"><span class="delimiter">&quot;</span><span class="content">myCF</span><span class="delimiter">&quot;</span></span>);
cfDesc.setPrefetchBlocksOnOpen(<span class="predefined-constant">true</span>);
tableDesc.addFamily(cfDesc);
<span class="comment">// ...</span></code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>See the API documentation for
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html">CacheConfig</a>.</p>
</div>
<div class="paragraph">
<p>To see prefetch in operation, enable TRACE level logging on
<code>org.apache.hadoop.hbase.io.hfile.HFileReaderImpl</code> in hbase-2.0+
or on <code>org.apache.hadoop.hbase.io.hfile.HFileReaderV2</code> in earlier versions, hbase-1.x, of HBase.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.rs.memstore.size"><a class="anchor" href="#perf.rs.memstore.size"></a>121.6. <code>hbase.regionserver.global.memstore.size</code></h3>
<div class="paragraph">
<p>See <a href="#hbase.regionserver.global.memstore.size">[hbase.regionserver.global.memstore.size]</a>.
This memory setting is often adjusted for the RegionServer process depending on needs.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.rs.memstore.size.lower.limit"><a class="anchor" href="#perf.rs.memstore.size.lower.limit"></a>121.7. <code>hbase.regionserver.global.memstore.size.lower.limit</code></h3>
<div class="paragraph">
<p>See <a href="#hbase.regionserver.global.memstore.size.lower.limit">[hbase.regionserver.global.memstore.size.lower.limit]</a>.
This memory setting is often adjusted for the RegionServer process depending on needs.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hstore.blockingstorefiles"><a class="anchor" href="#perf.hstore.blockingstorefiles"></a>121.8. <code>hbase.hstore.blockingStoreFiles</code></h3>
<div class="paragraph">
<p>See <a href="#hbase.hstore.blockingStoreFiles">[hbase.hstore.blockingStoreFiles]</a>.
If there is blocking in the RegionServer logs, increasing this can help.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hregion.memstore.block.multiplier"><a class="anchor" href="#perf.hregion.memstore.block.multiplier"></a>121.9. <code>hbase.hregion.memstore.block.multiplier</code></h3>
<div class="paragraph">
<p>See <a href="#hbase.hregion.memstore.block.multiplier">[hbase.hregion.memstore.block.multiplier]</a>.
If there is enough RAM, increasing this can help.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbase.regionserver.checksum.verify.performance"><a class="anchor" href="#hbase.regionserver.checksum.verify.performance"></a>121.10. <code>hbase.regionserver.checksum.verify</code></h3>
<div class="paragraph">
<p>Have HBase write the checksum into the datablock and save having to do the checksum seek whenever you read.</p>
</div>
<div class="paragraph">
<p>See <a href="#hbase.regionserver.checksum.verify">[hbase.regionserver.checksum.verify]</a>, <a href="#hbase.hstore.bytes.per.checksum">[hbase.hstore.bytes.per.checksum]</a> and <a href="#hbase.hstore.checksum.algorithm">[hbase.hstore.checksum.algorithm]</a>. For more information see the release note on <a href="https://issues.apache.org/jira/browse/HBASE-5074">HBASE-5074 support checksums in HBase block cache</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_tuning_code_callqueue_code_options"><a class="anchor" href="#_tuning_code_callqueue_code_options"></a>121.11. Tuning <code>callQueue</code> Options</h3>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-11355">HBASE-11355</a> introduces several callQueue tuning mechanisms which can increase performance.
See the JIRA for some benchmarking information.</p>
</div>
<div class="paragraph">
<p>To increase the number of callqueues, set <code>hbase.ipc.server.num.callqueue</code> to a value greater than <code>1</code>.
To split the callqueue into separate read and write queues, set <code>hbase.ipc.server.callqueue.read.ratio</code> to a value between <code>0</code> and <code>1</code>.
This factor weights the queues toward writes (if below .5) or reads (if above .5). Another way to say this is that the factor determines what percentage of the split queues are used for reads.
The following examples illustrate some of the possibilities.
Note that you always have at least one write queue, no matter what setting you use.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The default value of <code>0</code> does not split the queue.</p>
</li>
<li>
<p>A value of <code>.3</code> uses 30% of the queues for reading and 60% for writing.
Given a value of <code>10</code> for <code>hbase.ipc.server.num.callqueue</code>, 3 queues would be used for reads and 7 for writes.</p>
</li>
<li>
<p>A value of <code>.5</code> uses the same number of read queues and write queues.
Given a value of <code>10</code> for <code>hbase.ipc.server.num.callqueue</code>, 5 queues would be used for reads and 5 for writes.</p>
</li>
<li>
<p>A value of <code>.6</code> uses 60% of the queues for reading and 30% for reading.
Given a value of <code>10</code> for <code>hbase.ipc.server.num.callqueue</code>, 7 queues would be used for reads and 3 for writes.</p>
</li>
<li>
<p>A value of <code>1.0</code> uses one queue to process write requests, and all other queues process read requests.
A value higher than <code>1.0</code> has the same effect as a value of <code>1.0</code>.
Given a value of <code>10</code> for <code>hbase.ipc.server.num.callqueue</code>, 9 queues would be used for reads and 1 for writes.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>You can also split the read queues so that separate queues are used for short reads (from Get operations) and long reads (from Scan operations), by setting the <code>hbase.ipc.server.callqueue.scan.ratio</code> option.
This option is a factor between 0 and 1, which determine the ratio of read queues used for Gets and Scans.
More queues are used for Gets if the value is below <code>.5</code> and more are used for scans if the value is above <code>.5</code>.
No matter what setting you use, at least one read queue is used for Get operations.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A value of <code>0</code> does not split the read queue.</p>
</li>
<li>
<p>A value of <code>.3</code> uses 60% of the read queues for Gets and 30% for Scans.
Given a value of <code>20</code> for <code>hbase.ipc.server.num.callqueue</code> and a value of <code>.5</code> for <code>hbase.ipc.server.callqueue.read.ratio</code>, 10 queues would be used for reads, out of those 10, 7 would be used for Gets and 3 for Scans.</p>
</li>
<li>
<p>A value of <code>.5</code> uses half the read queues for Gets and half for Scans.
Given a value of <code>20</code> for <code>hbase.ipc.server.num.callqueue</code> and a value of <code>.5</code> for <code>hbase.ipc.server.callqueue.read.ratio</code>, 10 queues would be used for reads, out of those 10, 5 would be used for Gets and 5 for Scans.</p>
</li>
<li>
<p>A value of <code>.6</code> uses 30% of the read queues for Gets and 60% for Scans.
Given a value of <code>20</code> for <code>hbase.ipc.server.num.callqueue</code> and a value of <code>.5</code> for <code>hbase.ipc.server.callqueue.read.ratio</code>, 10 queues would be used for reads, out of those 10, 3 would be used for Gets and 7 for Scans.</p>
</li>
<li>
<p>A value of <code>1.0</code> uses all but one of the read queues for Scans.
Given a value of <code>20</code> for <code>hbase.ipc.server.num.callqueue</code> and a value of`.5` for <code>hbase.ipc.server.callqueue.read.ratio</code>, 10 queues would be used for reads, out of those 10, 1 would be used for Gets and 9 for Scans.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>You can use the new option <code>hbase.ipc.server.callqueue.handler.factor</code> to programmatically tune the number of queues:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A value of <code>0</code> uses a single shared queue between all the handlers.</p>
</li>
<li>
<p>A value of <code>1</code> uses a separate queue for each handler.</p>
</li>
<li>
<p>A value between <code>0</code> and <code>1</code> tunes the number of queues against the number of handlers.
For instance, a value of <code>.5</code> shares one queue between each two handlers.</p>
<div class="paragraph">
<p>Having more queues, such as in a situation where you have one queue per handler, reduces contention when adding a task to a queue or selecting it from a queue.
The trade-off is that if you have some queues with long-running tasks, a handler may end up waiting to execute from that queue rather than processing another queue which has waiting tasks.</p>
</div>
</li>
</ul>
</div>
<div class="paragraph">
<p>For these values to take effect on a given RegionServer, the RegionServer must be restarted.
These parameters are intended for testing purposes and should be used carefully.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.zookeeper"><a class="anchor" href="#perf.zookeeper"></a>122. ZooKeeper</h2>
<div class="sectionbody">
<div class="paragraph">
<p>See <a href="#zookeeper">ZooKeeper</a> for information on configuring ZooKeeper, and see the part about having a dedicated disk.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.schema"><a class="anchor" href="#perf.schema"></a>123. Schema Design</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="perf.number.of.cfs"><a class="anchor" href="#perf.number.of.cfs"></a>123.1. Number of Column Families</h3>
<div class="paragraph">
<p>See <a href="#number.of.cfs">On the number of column families</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.schema.keys"><a class="anchor" href="#perf.schema.keys"></a>123.2. Key and Attribute Lengths</h3>
<div class="paragraph">
<p>See <a href="#keysize">Try to minimize row and column sizes</a>.
See also <a href="#perf.compression.however">However&#8230;&#8203;</a> for compression caveats.</p>
</div>
</div>
<div class="sect2">
<h3 id="schema.regionsize"><a class="anchor" href="#schema.regionsize"></a>123.3. Table RegionSize</h3>
<div class="paragraph">
<p>The regionsize can be set on a per-table basis via <code>setFileSize</code> on <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html">HTableDescriptor</a> in the event where certain tables require different regionsizes than the configured default regionsize.</p>
</div>
<div class="paragraph">
<p>See <a href="#ops.capacity.regions">Determining region count and size</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="schema.bloom"><a class="anchor" href="#schema.bloom"></a>123.4. Bloom Filters</h3>
<div class="paragraph">
<p>A Bloom filter, named for its creator, Burton Howard Bloom, is a data structure which is designed to predict whether a given element is a member of a set of data.
A positive result from a Bloom filter is not always accurate, but a negative result is guaranteed to be accurate.
Bloom filters are designed to be "accurate enough" for sets of data which are so large that conventional hashing mechanisms would be impractical.
For more information about Bloom filters in general, refer to <a href="http://en.wikipedia.org/wiki/Bloom_filter" class="bare">http://en.wikipedia.org/wiki/Bloom_filter</a>.</p>
</div>
<div class="paragraph">
<p>In terms of HBase, Bloom filters provide a lightweight in-memory structure to reduce the number of disk reads for a given Get operation (Bloom filters do not work with Scans) to only the StoreFiles likely to contain the desired Row.
The potential performance gain increases with the number of parallel reads.</p>
</div>
<div class="paragraph">
<p>The Bloom filters themselves are stored in the metadata of each HFile and never need to be updated.
When an HFile is opened because a region is deployed to a RegionServer, the Bloom filter is loaded into memory.</p>
</div>
<div class="paragraph">
<p>HBase includes some tuning mechanisms for folding the Bloom filter to reduce the size and keep the false positive rate within a desired range.</p>
</div>
<div class="paragraph">
<p>Bloom filters were introduced in <a href="https://issues.apache.org/jira/browse/HBASE-1200">HBASE-1200</a>.
Since HBase 0.96, row-based Bloom filters are enabled by default.
(<a href="https://issues.apache.org/jira/browse/HBASE-8450">HBASE-8450</a>)</p>
</div>
<div class="paragraph">
<p>For more information on Bloom filters in relation to HBase, see <a href="#blooms">Bloom Filters</a> for more information, or the following Quora discussion: <a href="http://www.quora.com/How-are-bloom-filters-used-in-HBase">How are bloom filters used in HBase?</a>.</p>
</div>
<div class="sect3">
<h4 id="bloom.filters.when"><a class="anchor" href="#bloom.filters.when"></a>123.4.1. When To Use Bloom Filters</h4>
<div class="paragraph">
<p>Since HBase 0.96, row-based Bloom filters are enabled by default.
You may choose to disable them or to change some tables to use row+column Bloom filters, depending on the characteristics of your data and how it is loaded into HBase.</p>
</div>
<div class="paragraph">
<p>To determine whether Bloom filters could have a positive impact, check the value of <code>blockCacheHitRatio</code> in the RegionServer metrics.
If Bloom filters are enabled, the value of <code>blockCacheHitRatio</code> should increase, because the Bloom filter is filtering out blocks that are definitely not needed.</p>
</div>
<div class="paragraph">
<p>You can choose to enable Bloom filters for a row or for a row+column combination.
If you generally scan entire rows, the row+column combination will not provide any benefit.
A row-based Bloom filter can operate on a row+column Get, but not the other way around.
However, if you have a large number of column-level Puts, such that a row may be present in every StoreFile, a row-based filter will always return a positive result and provide no benefit.
Unless you have one column per row, row+column Bloom filters require more space, in order to store more keys.
Bloom filters work best when the size of each data entry is at least a few kilobytes in size.</p>
</div>
<div class="paragraph">
<p>Overhead will be reduced when your data is stored in a few larger StoreFiles, to avoid extra disk IO during low-level scans to find a specific row.</p>
</div>
<div class="paragraph">
<p>Bloom filters need to be rebuilt upon deletion, so may not be appropriate in environments with a large number of deletions.</p>
</div>
</div>
<div class="sect3">
<h4 id="_enabling_bloom_filters"><a class="anchor" href="#_enabling_bloom_filters"></a>123.4.2. Enabling Bloom Filters</h4>
<div class="paragraph">
<p>Bloom filters are enabled on a Column Family.
You can do this by using the setBloomFilterType method of HColumnDescriptor or using the HBase API.
Valid values are <code>NONE</code>, <code>ROW</code> (default), or <code>ROWCOL</code>.
See <a href="#bloom.filters.when">When To Use Bloom Filters</a> for more information on <code>ROW</code> versus <code>ROWCOL</code>.
See also the API documentation for <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a>.</p>
</div>
<div class="paragraph">
<p>The following example creates a table and enables a ROWCOL Bloom filter on the <code>colfam1</code> column family.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; create 'mytable',{NAME =&gt; 'colfam1', BLOOMFILTER =&gt; 'ROWCOL'}</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_configuring_server_wide_behavior_of_bloom_filters"><a class="anchor" href="#_configuring_server_wide_behavior_of_bloom_filters"></a>123.4.3. Configuring Server-Wide Behavior of Bloom Filters</h4>
<div class="paragraph">
<p>You can configure the following settings in the <em>hbase-site.xml</em>.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Parameter</th>
<th class="tableblock halign-left valign-top">Default</th>
<th class="tableblock halign-left valign-top">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.bloom.enabled</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">yes</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Set to no to kill bloom filters server-wide if something goes wrong</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.bloom.error.rate</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">.01</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The average false positive rate for bloom filters. Folding is used to
maintain the false positive rate. Expressed as a decimal representation of a
percentage.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.bloom.max.fold</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">7</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The guaranteed maximum fold rate. Changing this setting should not be
necessary and is not recommended.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.bloom.max.keys</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">128000000</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">For default (single-block) Bloom filters, this specifies the maximum number of keys.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.delete.family.bloom.enabled</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">true</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Master switch to enable Delete Family Bloom filters and store them in the StoreFile.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">io.storefile.bloom.block.size</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">131072</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Target Bloom block size. Bloom filter blocks of approximately this size
are interleaved with data blocks.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.block.bloom.cacheonwrite</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">false</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Enables cache-on-write for inline blocks of a compound Bloom filter.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect2">
<h3 id="schema.cf.blocksize"><a class="anchor" href="#schema.cf.blocksize"></a>123.5. ColumnFamily BlockSize</h3>
<div class="paragraph">
<p>The blocksize can be configured for each ColumnFamily in a table, and defaults to 64k.
Larger cell values require larger blocksizes.
There is an inverse relationship between blocksize and the resulting StoreFile indexes (i.e., if the blocksize is doubled then the resulting indexes should be roughly halved).</p>
</div>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a> and <a href="#store">Store</a>for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="cf.in.memory"><a class="anchor" href="#cf.in.memory"></a>123.6. In-Memory ColumnFamilies</h3>
<div class="paragraph">
<p>ColumnFamilies can optionally be defined as in-memory.
Data is still persisted to disk, just like any other ColumnFamily.
In-memory blocks have the highest priority in the <a href="#block.cache">Block Cache</a>, but it is not a guarantee that the entire table will be in memory.</p>
</div>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.compression"><a class="anchor" href="#perf.compression"></a>123.7. Compression</h3>
<div class="paragraph">
<p>Production systems should use compression with their ColumnFamily definitions.
See <a href="#compression">Compression and Data Block Encoding In HBase</a> for more information.</p>
</div>
<div class="sect3">
<h4 id="perf.compression.however"><a class="anchor" href="#perf.compression.however"></a>123.7.1. However&#8230;&#8203;</h4>
<div class="paragraph">
<p>Compression deflates data <em>on disk</em>.
When it&#8217;s in-memory (e.g., in the MemStore) or on the wire (e.g., transferring between RegionServer and Client) it&#8217;s inflated.
So while using ColumnFamily compression is a best practice, but it&#8217;s not going to completely eliminate the impact of over-sized Keys, over-sized ColumnFamily names, or over-sized Column names.</p>
</div>
<div class="paragraph">
<p>See <a href="#keysize">Try to minimize row and column sizes</a> on for schema design tips, and <a href="#keyvalue">KeyValue</a> for more information on HBase stores data internally.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.general"><a class="anchor" href="#perf.general"></a>124. HBase General Patterns</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="perf.general.constants"><a class="anchor" href="#perf.general.constants"></a>124.1. Constants</h3>
<div class="paragraph">
<p>When people get started with HBase they have a tendency to write code that looks like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Get get = <span class="keyword">new</span> Get(rowkey);
<span class="predefined-type">Result</span> r = table.get(get);
<span class="type">byte</span><span class="type">[]</span> b = r.getValue(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>)); <span class="comment">// returns current version of value</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>But especially when inside loops (and MapReduce jobs), converting the columnFamily and column-names to byte-arrays repeatedly is surprisingly expensive.
It&#8217;s better to use constants for the byte-arrays, like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">cf</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> ATTR = <span class="string"><span class="delimiter">&quot;</span><span class="content">attr</span><span class="delimiter">&quot;</span></span>.getBytes();
...
Get get = <span class="keyword">new</span> Get(rowkey);
<span class="predefined-type">Result</span> r = table.get(get);
<span class="type">byte</span><span class="type">[]</span> b = r.getValue(CF, ATTR); <span class="comment">// returns current version of value</span></code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.writing"><a class="anchor" href="#perf.writing"></a>125. Writing to HBase</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="perf.batch.loading"><a class="anchor" href="#perf.batch.loading"></a>125.1. Batch Loading</h3>
<div class="paragraph">
<p>Use the bulk load tool if you can.
See <a href="#arch.bulk.load">Bulk Loading</a>.
Otherwise, pay attention to the below.</p>
</div>
</div>
<div class="sect2">
<h3 id="precreate.regions"><a class="anchor" href="#precreate.regions"></a>125.2. Table Creation: Pre-Creating Regions</h3>
<div class="paragraph">
<p>Tables in HBase are initially created with one region by default.
For bulk imports, this means that all clients will write to the same region until it is large enough to split and become distributed across the cluster.
A useful pattern to speed up the bulk import process is to pre-create empty regions.
Be somewhat conservative in this, because too-many regions can actually degrade performance.</p>
</div>
<div class="paragraph">
<p>There are two different approaches to pre-creating splits using the HBase API.
The first approach is to rely on the default <code>Admin</code> strategy (which is implemented in <code>Bytes.split</code>)&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="type">byte</span><span class="type">[]</span> startKey = ...; <span class="comment">// your lowest key</span>
<span class="type">byte</span><span class="type">[]</span> endKey = ...; <span class="comment">// your highest key</span>
<span class="type">int</span> numberOfRegions = ...; <span class="comment">// # of regions to create</span>
admin.createTable(table, startKey, endKey, numberOfRegions);</code></pre>
</div>
</div>
<div class="paragraph">
<p>And the other approach, using the HBase API, is to define the splits yourself&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="type">byte</span><span class="type">[]</span><span class="type">[]</span> splits = ...; <span class="comment">// create your own splits</span>
admin.createTable(table, splits);</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can achieve a similar effect using the HBase Shell to create tables by specifying split options.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">#</span> create table with specific split points
hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t1</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>,SPLITS =&gt; [<span class="string"><span class="delimiter">'</span><span class="content">\x10</span><span class="content">\x00</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">\x20</span><span class="content">\x00</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">\x30</span><span class="content">\x00</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">\x40</span><span class="content">\x00</span><span class="delimiter">'</span></span>]
<span class="error">#</span> create table with four regions based on random bytes keys
hbase&gt;create <span class="string"><span class="delimiter">'</span><span class="content">t2</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>, { NUMREGIONS =&gt; <span class="integer">4</span> , SPLITALGO =&gt; <span class="string"><span class="delimiter">'</span><span class="content">UniformSplit</span><span class="delimiter">'</span></span> }
<span class="error">#</span> create table with five regions based on hex keys
create <span class="string"><span class="delimiter">'</span><span class="content">t3</span><span class="delimiter">'</span></span>,<span class="string"><span class="delimiter">'</span><span class="content">f1</span><span class="delimiter">'</span></span>, { NUMREGIONS =&gt; <span class="integer">5</span>, SPLITALGO =&gt; <span class="string"><span class="delimiter">'</span><span class="content">HexStringSplit</span><span class="delimiter">'</span></span> }</code></pre>
</div>
</div>
<div class="paragraph">
<p>See <a href="#rowkey.regionsplits">Relationship Between RowKeys and Region Splits</a> for issues related to understanding your keyspace and pre-creating regions.
See <a href="#manual_region_splitting_decisions">manual region splitting decisions</a> for discussion on manually pre-splitting regions.
See <a href="#tricks.pre-split">Pre-splitting tables with the HBase Shell</a> for more details of using the HBase Shell to pre-split tables.</p>
</div>
</div>
<div class="sect2">
<h3 id="def.log.flush"><a class="anchor" href="#def.log.flush"></a>125.3. Table Creation: Deferred Log Flush</h3>
<div class="paragraph">
<p>The default behavior for Puts using the Write Ahead Log (WAL) is that <code>WAL</code> edits will be written immediately.
If deferred log flush is used, WAL edits are kept in memory until the flush period.
The benefit is aggregated and asynchronous <code>WAL</code>- writes, but the potential downside is that if the RegionServer goes down the yet-to-be-flushed edits are lost.
This is safer, however, than not using WAL at all with Puts.</p>
</div>
<div class="paragraph">
<p>Deferred log flush can be configured on tables via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html">HTableDescriptor</a>.
The default value of <code>hbase.regionserver.optionallogflushinterval</code> is 1000ms.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.putwal"><a class="anchor" href="#perf.hbase.client.putwal"></a>125.4. HBase Client: Turn off WAL on Puts</h3>
<div class="paragraph">
<p>A frequent request is to disable the WAL to increase performance of Puts.
This is only appropriate for bulk loads, as it puts your data at risk by removing the protection of the WAL in the event of a region server crash.
Bulk loads can be re-run in the event of a crash, with little risk of data loss.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
If you disable the WAL for anything other than bulk loads, your data is at risk.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>In general, it is best to use WAL for Puts, and where loading throughput is a concern to use bulk loading techniques instead.
For normal Puts, you are not likely to see a performance improvement which would outweigh the risk.
To disable the WAL, see <a href="#wal.disable">Disabling the WAL</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.regiongroup"><a class="anchor" href="#perf.hbase.client.regiongroup"></a>125.5. HBase Client: Group Puts by RegionServer</h3>
<div class="paragraph">
<p>In addition to using the writeBuffer, grouping <code>Put`s by RegionServer can reduce the number of client RPC calls per writeBuffer flush.
There is a utility `HTableUtil</code> currently on MASTER that does this, but you can either copy that or implement your own version for those still on 0.90.x or earlier.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.write.mr.reducer"><a class="anchor" href="#perf.hbase.write.mr.reducer"></a>125.6. MapReduce: Skip The Reducer</h3>
<div class="paragraph">
<p>When writing a lot of data to an HBase table from a MR job (e.g., with <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html">TableOutputFormat</a>), and specifically where Puts are being emitted from the Mapper, skip the Reducer step.
When a Reducer step is used, all of the output (Puts) from the Mapper will get spooled to disk, then sorted/shuffled to other Reducers that will most likely be off-node.
It&#8217;s far more efficient to just write directly to HBase.</p>
</div>
<div class="paragraph">
<p>For summary jobs where HBase is used as a source and a sink, then writes will be coming from the Reducer step (e.g., summarize values then write out result). This is a different processing problem than from the above case.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.one.region"><a class="anchor" href="#perf.one.region"></a>125.7. Anti-Pattern: One Hot Region</h3>
<div class="paragraph">
<p>If all your data is being written to one region at a time, then re-read the section on processing timeseries data.</p>
</div>
<div class="paragraph">
<p>Also, if you are pre-splitting regions and all your data is <em>still</em> winding up in a single region even though your keys aren&#8217;t monotonically increasing, confirm that your keyspace actually works with the split strategy.
There are a variety of reasons that regions may appear "well split" but won&#8217;t work with your data.
As the HBase client communicates directly with the RegionServers, this can be obtained via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/RegionLocator.html#getRegionLocation-byte:A-">RegionLocator.getRegionLocation</a>.</p>
</div>
<div class="paragraph">
<p>See <a href="#precreate.regions">Table Creation: Pre-Creating Regions</a>, as well as <a href="#perf.configurations">HBase Configurations</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.reading"><a class="anchor" href="#perf.reading"></a>126. Reading from HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The mailing list can help if you are having performance issues.
For example, here is a good general thread on what to look at addressing read-time issues: <a href="http://search-hadoop.com/m/qOo2yyHtCC1">HBase Random Read latency &gt; 100ms</a></p>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.caching"><a class="anchor" href="#perf.hbase.client.caching"></a>126.1. Scan Caching</h3>
<div class="paragraph">
<p>If HBase is used as an input source for a MapReduce job, for example, make sure that the input <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> instance to the MapReduce job has <code>setCaching</code> set to something greater than the default (which is 1). Using the default value means that the map-task will make call back to the region-server for every record processed.
Setting this value to 500, for example, will transfer 500 rows at a time to the client to be processed.
There is a cost/benefit to have the cache value be large because it costs more in memory for both client and RegionServer, so bigger isn&#8217;t always better.</p>
</div>
<div class="sect3">
<h4 id="perf.hbase.client.caching.mr"><a class="anchor" href="#perf.hbase.client.caching.mr"></a>126.1.1. Scan Caching in MapReduce Jobs</h4>
<div class="paragraph">
<p>Scan settings in MapReduce jobs deserve special attention.
Timeouts can result (e.g., UnknownScannerException) in Map tasks if it takes longer to process a batch of records before the client goes back to the RegionServer for the next set of data.
This problem can occur because there is non-trivial processing occurring per row.
If you process rows quickly, set caching higher.
If you process rows more slowly (e.g., lots of transformations per row, writes), then set caching lower.</p>
</div>
<div class="paragraph">
<p>Timeouts can also happen in a non-MapReduce use case (i.e., single threaded HBase client doing a Scan), but the processing that is often performed in MapReduce jobs tends to exacerbate this issue.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.selection"><a class="anchor" href="#perf.hbase.client.selection"></a>126.2. Scan Attribute Selection</h3>
<div class="paragraph">
<p>Whenever a Scan is used to process large numbers of rows (and especially when used as a MapReduce source), be aware of which attributes are selected.
If <code>scan.addFamily</code> is called then <em>all</em> of the attributes in the specified ColumnFamily will be returned to the client.
If only a small number of the available attributes are to be processed, then only those attributes should be specified in the input scan because attribute over-selection is a non-trivial performance penalty over large datasets.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.seek"><a class="anchor" href="#perf.hbase.client.seek"></a>126.3. Avoid scan seeks</h3>
<div class="paragraph">
<p>When columns are selected explicitly with <code>scan.addColumn</code>, HBase will schedule seek operations to seek between the selected columns.
When rows have few columns and each column has only a few versions this can be inefficient.
A seek operation is generally slower if does not seek at least past 5-10 columns/versions or 512-1024 bytes.</p>
</div>
<div class="paragraph">
<p>In order to opportunistically look ahead a few columns/versions to see if the next column/version can be found that way before a seek operation is scheduled, a new attribute <code>Scan.HINT_LOOKAHEAD</code> can be set on the Scan object.
The following code instructs the RegionServer to attempt two iterations of next before a seek is scheduled:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Scan scan = <span class="keyword">new</span> Scan();
scan.addColumn(...);
scan.setAttribute(Scan.HINT_LOOKAHEAD, Bytes.toBytes(<span class="integer">2</span>));
table.getScanner(scan);</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.mr.input"><a class="anchor" href="#perf.hbase.mr.input"></a>126.4. MapReduce - Input Splits</h3>
<div class="paragraph">
<p>For MapReduce jobs that use HBase tables as a source, if there a pattern where the "slow" map tasks seem to have the same Input Split (i.e., the RegionServer serving the data), see the Troubleshooting Case Study in <a href="#casestudies.slownode">Case Study #1 (Performance Issue On A Single Node)</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.scannerclose"><a class="anchor" href="#perf.hbase.client.scannerclose"></a>126.5. Close ResultScanners</h3>
<div class="paragraph">
<p>This isn&#8217;t so much about improving performance but rather <em>avoiding</em> performance problems.
If you forget to close <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/ResultScanner.html">ResultScanners</a> you can cause problems on the RegionServers.
Always have ResultScanner processing enclosed in try/catch blocks.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Scan scan = <span class="keyword">new</span> Scan();
<span class="comment">// set attrs...</span>
ResultScanner rs = table.getScanner(scan);
<span class="keyword">try</span> {
<span class="keyword">for</span> (<span class="predefined-type">Result</span> r = rs.next(); r != <span class="predefined-constant">null</span>; r = rs.next()) {
<span class="comment">// process result...</span>
} <span class="keyword">finally</span> {
rs.close(); <span class="comment">// always close the ResultScanner!</span>
}
table.close();</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.blockcache"><a class="anchor" href="#perf.hbase.client.blockcache"></a>126.6. Block Cache</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</a> instances can be set to use the block cache in the RegionServer via the <code>setCacheBlocks</code> method.
For input Scans to MapReduce jobs, this should be <code>false</code>.
For frequently accessed rows, it is advisable to use the block cache.</p>
</div>
<div class="paragraph">
<p>Cache more data by moving your Block Cache off-heap.
See <a href="#offheap.blockcache">Off-heap Block Cache</a></p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.client.rowkeyonly"><a class="anchor" href="#perf.hbase.client.rowkeyonly"></a>126.7. Optimal Loading of Row Keys</h3>
<div class="paragraph">
<p>When performing a table <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">scan</a> where only the row keys are needed (no families, qualifiers, values or timestamps), add a FilterList with a <code>MUST_PASS_ALL</code> operator to the scanner using <code>setFilter</code>.
The filter list should include both a <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html">FirstKeyOnlyFilter</a> and a <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/KeyOnlyFilter.html">KeyOnlyFilter</a>.
Using this filter combination will result in a worst case scenario of a RegionServer reading a single value from disk and minimal network traffic to the client for a single row.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hbase.read.dist"><a class="anchor" href="#perf.hbase.read.dist"></a>126.8. Concurrency: Monitor Data Spread</h3>
<div class="paragraph">
<p>When performing a high number of concurrent reads, monitor the data spread of the target tables.
If the target table(s) have too few regions then the reads could likely be served from too few nodes.</p>
</div>
<div class="paragraph">
<p>See <a href="#precreate.regions">Table Creation: Pre-Creating Regions</a>, as well as <a href="#perf.configurations">HBase Configurations</a></p>
</div>
</div>
<div class="sect2">
<h3 id="blooms"><a class="anchor" href="#blooms"></a>126.9. Bloom Filters</h3>
<div class="paragraph">
<p>Enabling Bloom Filters can save your having to go to disk and can help improve read latencies.</p>
</div>
<div class="paragraph">
<p><a href="http://en.wikipedia.org/wiki/Bloom_filter">Bloom filters</a> were developed over in <a href="https://issues.apache.org/jira/browse/HBASE-1200">HBase-1200 Add bloomfilters</a>.
For description of the development process&#8201;&#8212;&#8201;why static blooms rather than dynamic&#8201;&#8212;&#8201;and for an overview of the unique properties that pertain to blooms in HBase, as well as possible future directions, see the <em>Development Process</em> section of the document <a href="https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf">BloomFilters in HBase</a> attached to <a href="https://issues.apache.org/jira/browse/HBASE-1200">HBASE-1200</a>.
The bloom filters described here are actually version two of blooms in HBase.
In versions up to 0.19.x, HBase had a dynamic bloom option based on work done by the <a href="http://www.onelab.org">European Commission One-Lab Project 034819</a>.
The core of the HBase bloom work was later pulled up into Hadoop to implement org.apache.hadoop.io.BloomMapFile.
Version 1 of HBase blooms never worked that well.
Version 2 is a rewrite from scratch though again it starts with the one-lab work.</p>
</div>
<div class="paragraph">
<p>See also <a href="#schema.bloom">Bloom Filters</a>.</p>
</div>
<div class="sect3">
<h4 id="bloom_footprint"><a class="anchor" href="#bloom_footprint"></a>126.9.1. Bloom StoreFile footprint</h4>
<div class="paragraph">
<p>Bloom filters add an entry to the <code>StoreFile</code> general <code>FileInfo</code> data structure and then two extra entries to the <code>StoreFile</code> metadata section.</p>
</div>
<div class="sect4">
<h5 id="_bloomfilter_in_the_code_storefile_fileinfo_code_data_structure"><a class="anchor" href="#_bloomfilter_in_the_code_storefile_fileinfo_code_data_structure"></a>BloomFilter in the <code>StoreFile``FileInfo</code> data structure</h5>
<div class="paragraph">
<p><code>FileInfo</code> has a <code>BLOOM_FILTER_TYPE</code> entry which is set to <code>NONE</code>, <code>ROW</code> or <code>ROWCOL.</code></p>
</div>
</div>
<div class="sect4">
<h5 id="_bloomfilter_entries_in_code_storefile_code_metadata"><a class="anchor" href="#_bloomfilter_entries_in_code_storefile_code_metadata"></a>BloomFilter entries in <code>StoreFile</code> metadata</h5>
<div class="paragraph">
<p><code>BLOOM_FILTER_META</code> holds Bloom Size, Hash Function used, etc.
It&#8217;s small in size and is cached on <code>StoreFile.Reader</code> load</p>
</div>
<div class="paragraph">
<p><code>BLOOM_FILTER_DATA</code> is the actual bloomfilter data.
Obtained on-demand.
Stored in the LRU cache, if it is enabled (It&#8217;s enabled by default).</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="config.bloom"><a class="anchor" href="#config.bloom"></a>126.9.2. Bloom Filter Configuration</h4>
<div class="sect4">
<h5 id="__code_io_storefile_bloom_enabled_code_global_kill_switch"><a class="anchor" href="#__code_io_storefile_bloom_enabled_code_global_kill_switch"></a><code>io.storefile.bloom.enabled</code> global kill switch</h5>
<div class="paragraph">
<p><code>io.storefile.bloom.enabled</code> in <code>Configuration</code> serves as the kill switch in case something goes wrong.
Default = <code>true</code>.</p>
</div>
</div>
<div class="sect4">
<h5 id="__code_io_storefile_bloom_error_rate_code"><a class="anchor" href="#__code_io_storefile_bloom_error_rate_code"></a><code>io.storefile.bloom.error.rate</code></h5>
<div class="paragraph">
<p><code>io.storefile.bloom.error.rate</code> = average false positive rate.
Default = 1%. Decrease rate by ½ (e.g.
to .5%) == +1 bit per bloom entry.</p>
</div>
</div>
<div class="sect4">
<h5 id="__code_io_storefile_bloom_max_fold_code"><a class="anchor" href="#__code_io_storefile_bloom_max_fold_code"></a><code>io.storefile.bloom.max.fold</code></h5>
<div class="paragraph">
<p><code>io.storefile.bloom.max.fold</code> = guaranteed minimum fold rate.
Most people should leave this alone.
Default = 7, or can collapse to at least 1/128th of original size.
See the <em>Development Process</em> section of the document <a href="https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf">BloomFilters in HBase</a> for more on what this option means.</p>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hedged.reads"><a class="anchor" href="#hedged.reads"></a>126.10. Hedged Reads</h3>
<div class="paragraph">
<p>Hedged reads are a feature of HDFS, introduced in Hadoop 2.4.0 with <a href="https://issues.apache.org/jira/browse/HDFS-5776">HDFS-5776</a>.
Normally, a single thread is spawned for each read request.
However, if hedged reads are enabled, the client waits some
configurable amount of time, and if the read does not return,
the client spawns a second read request, against a different
block replica of the same data. Whichever read returns first is
used, and the other read request is discarded.</p>
</div>
<div class="paragraph">
<p>Hedged reads are "&#8230;&#8203;very good at eliminating outlier datanodes, which
in turn makes them very good choice for latency sensitive setups.
But, if you are looking for maximizing throughput, hedged reads tend to
create load amplification as things get slower in general. In short,
the thing to watch out for is the non-graceful performance degradation
when you are running close a certain throughput threshold." (Quote from Ashu Pachauri in HBASE-17083).</p>
</div>
<div class="paragraph">
<p>Other concerns to keep in mind while running with hedged reads enabled
include:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>They may lead to network congestion. See <a href="https://issues.apache.org/jira/browse/HBASE-17083">HBASE-17083</a></p>
</li>
<li>
<p>Make sure you set the thread pool large enough so as blocking on the pool does not become a bottleneck (Again see <a href="https://issues.apache.org/jira/browse/HBASE-17083">HBASE-17083</a>)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>(From Yu Li up in HBASE-17083)</p>
</div>
<div class="paragraph">
<p>Because an HBase RegionServer is a HDFS client, you can enable hedged
reads in HBase, by adding the following properties to the RegionServer&#8217;s
hbase-site.xml and tuning the values to suit your environment.</p>
</div>
<div class="ulist">
<div class="title">Configuration for Hedged Reads</div>
<ul>
<li>
<p><code>dfs.client.hedged.read.threadpool.size</code> - the number of threads dedicated to servicing hedged reads.
If this is set to 0 (the default), hedged reads are disabled.</p>
</li>
<li>
<p><code>dfs.client.hedged.read.threshold.millis</code> - the number of milliseconds to wait before spawning a second read thread.</p>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 42. Hedged Reads Configuration Example</div>
<div class="content">
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.client.hedged.read.threadpool.size<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>20<span class="tag">&lt;/value&gt;</span> <span class="comment">&lt;!-- 20 threads --&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.client.hedged.read.threshold.millis<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>10<span class="tag">&lt;/value&gt;</span> <span class="comment">&lt;!-- 10 milliseconds --&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="paragraph">
<p>Use the following metrics to tune the settings for hedged reads on your cluster.
See <a href="#hbase_metrics">HBase Metrics</a> for more information.</p>
</div>
<div class="ulist">
<div class="title">Metrics for Hedged Reads</div>
<ul>
<li>
<p>hedgedReadOps - the number of times hedged read threads have been triggered.
This could indicate that read requests are often slow, or that hedged reads are triggered too quickly.</p>
</li>
<li>
<p>hedgeReadOpsWin - the number of times the hedged read thread was faster than the original thread.
This could indicate that a given RegionServer is having trouble servicing requests.</p>
</li>
</ul>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.deleting"><a class="anchor" href="#perf.deleting"></a>127. Deleting from HBase</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="perf.deleting.queue"><a class="anchor" href="#perf.deleting.queue"></a>127.1. Using HBase Tables as Queues</h3>
<div class="paragraph">
<p>HBase tables are sometimes used as queues.
In this case, special care must be taken to regularly perform major compactions on tables used in this manner.
As is documented in <a href="#datamodel">Data Model</a>, marking rows as deleted creates additional StoreFiles which then need to be processed on reads.
Tombstones only get cleaned up with major compactions.</p>
</div>
<div class="paragraph">
<p>See also <a href="#compaction">Compaction</a> and <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact-org.apache.hadoop.hbase.TableName-">Admin.majorCompact</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.deleting.rpc"><a class="anchor" href="#perf.deleting.rpc"></a>127.2. Delete RPC Behavior</h3>
<div class="paragraph">
<p>Be aware that <code>Table.delete(Delete)</code> doesn&#8217;t use the writeBuffer.
It will execute an RegionServer RPC with each invocation.
For a large number of deletes, consider <code>Table.delete(List)</code>.</p>
</div>
<div class="paragraph">
<p>See <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete-org.apache.hadoop.hbase.client.Delete-">hbase.client.Delete</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.hdfs"><a class="anchor" href="#perf.hdfs"></a>128. HDFS</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Because HBase runs on <a href="#arch.hdfs">HDFS</a> it is important to understand how it works and how it affects HBase.</p>
</div>
<div class="sect2">
<h3 id="perf.hdfs.curr"><a class="anchor" href="#perf.hdfs.curr"></a>128.1. Current Issues With Low-Latency Reads</h3>
<div class="paragraph">
<p>The original use-case for HDFS was batch processing.
As such, there low-latency reads were historically not a priority.
With the increased adoption of Apache HBase this is changing, and several improvements are already in development.
See the <a href="https://issues.apache.org/jira/browse/HDFS-1599">Umbrella Jira Ticket for HDFS Improvements for HBase</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="perf.hdfs.configs.localread"><a class="anchor" href="#perf.hdfs.configs.localread"></a>128.2. Leveraging local data</h3>
<div class="paragraph">
<p>Since Hadoop 1.0.0 (also 0.22.1, 0.23.1, CDH3u3 and HDP 1.0) via <a href="https://issues.apache.org/jira/browse/HDFS-2246">HDFS-2246</a>, it is possible for the DFSClient to take a "short circuit" and read directly from the disk instead of going through the DataNode when the data is local.
What this means for HBase is that the RegionServers can read directly off their machine&#8217;s disks instead of having to open a socket to talk to the DataNode, the former being generally much faster.
See JD&#8217;s <a href="http://files.meetup.com/1350427/hug_ebay_jdcryans.pdf">Performance Talk</a>.
Also see <a href="http://search-hadoop.com/m/zV6dKrLCVh1">HBase, mail # dev - read short circuit</a> thread for more discussion around short circuit reads.</p>
</div>
<div class="paragraph">
<p>To enable "short circuit" reads, it will depend on your version of Hadoop.
The original shortcircuit read patch was much improved upon in Hadoop 2 in <a href="https://issues.apache.org/jira/browse/HDFS-347">HDFS-347</a>.
See <a href="http://blog.cloudera.com/blog/2013/08/how-improved-short-circuit-local-reads-bring-better-performance-and-security-to-hadoop/" class="bare">http://blog.cloudera.com/blog/2013/08/how-improved-short-circuit-local-reads-bring-better-performance-and-security-to-hadoop/</a> for details on the difference between the old and new implementations.
See <a href="http://archive.cloudera.com/cdh4/cdh/4/hadoop/hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html">Hadoop shortcircuit reads configuration page</a> for how to enable the latter, better version of shortcircuit.
For example, here is a minimal config.
enabling short-circuit reads added to <em>hbase-site.xml</em>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.client.read.shortcircuit<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
This configuration parameter turns on short-circuit local reads.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>dfs.domain.socket.path<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/home/stack/sockets/short_circuit_read_socket_PORT<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;description&gt;</span>
Optional. This is a path to a UNIX domain socket that will be used for
communication between the DataNode and local HDFS clients.
If the string &quot;_PORT&quot; is present in this path, it will be replaced by the
TCP port of the DataNode.
<span class="tag">&lt;/description&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Be careful about permissions for the directory that hosts the shared domain socket; dfsclient will complain if open to other than the hbase user.</p>
</div>
<div class="paragraph">
<p>If you are running on an old Hadoop, one that is without <a href="https://issues.apache.org/jira/browse/HDFS-347">HDFS-347</a> but that has <a href="https://issues.apache.org/jira/browse/HDFS-2246">HDFS-2246</a>, you must set two configurations.
First, the hdfs-site.xml needs to be amended.
Set the property <code>dfs.block.local-path-access.user</code> to be the <em>only</em> user that can use the shortcut.
This has to be the user that started HBase.
Then in hbase-site.xml, set <code>dfs.client.read.shortcircuit</code> to be <code>true</code></p>
</div>
<div class="paragraph">
<p>Services&#8201;&#8212;&#8201;at least the HBase RegionServers&#8201;&#8212;&#8201;will need to be restarted in order to pick up the new configurations.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">dfs.client.read.shortcircuit.buffer.size</div>
<div class="paragraph">
<p>The default for this value is too high when running on a highly trafficked HBase.
In HBase, if this value has not been set, we set it down from the default of 1M to 128k (Since HBase 0.98.0 and 0.96.1). See <a href="https://issues.apache.org/jira/browse/HBASE-8143">HBASE-8143 HBase on Hadoop 2 with local short circuit reads (ssr) causes OOM</a>). The Hadoop DFSClient in HBase will allocate a direct byte buffer of this size for <em>each</em> block it has open; given HBase keeps its HDFS files open all the time, this can add up quickly.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
<div class="sect2">
<h3 id="perf.hdfs.comp"><a class="anchor" href="#perf.hdfs.comp"></a>128.3. Performance Comparisons of HBase vs. HDFS</h3>
<div class="paragraph">
<p>A fairly common question on the dist-list is why HBase isn&#8217;t as performant as HDFS files in a batch context (e.g., as a MapReduce source or sink). The short answer is that HBase is doing a lot more than HDFS (e.g., reading the KeyValues, returning the most current row or specified timestamps, etc.), and as such HBase is 4-5 times slower than HDFS in this processing context.
There is room for improvement and this gap will, over time, be reduced, but HDFS will always be faster in this use-case.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.ec2"><a class="anchor" href="#perf.ec2"></a>129. Amazon EC2</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Performance questions are common on Amazon EC2 environments because it is a shared environment.
You will not see the same throughput as a dedicated server.
In terms of running tests on EC2, run them several times for the same reason (i.e., it&#8217;s a shared environment and you don&#8217;t know what else is happening on the server).</p>
</div>
<div class="paragraph">
<p>If you are running on EC2 and post performance questions on the dist-list, please state this fact up-front that because EC2 issues are practically a separate class of performance issues.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.hbase.mr.cluster"><a class="anchor" href="#perf.hbase.mr.cluster"></a>130. Collocating HBase and MapReduce</h2>
<div class="sectionbody">
<div class="paragraph">
<p>It is often recommended to have different clusters for HBase and MapReduce.
A better qualification of this is: don&#8217;t collocate an HBase that serves live requests with a heavy MR workload.
OLTP and OLAP-optimized systems have conflicting requirements and one will lose to the other, usually the former.
For example, short latency-sensitive disk reads will have to wait in line behind longer reads that are trying to squeeze out as much throughput as possible.
MR jobs that write to HBase will also generate flushes and compactions, which will in turn invalidate blocks in the <a href="#block.cache">Block Cache</a>.</p>
</div>
<div class="paragraph">
<p>If you need to process the data from your live HBase cluster in MR, you can ship the deltas with <a href="#copy.table">CopyTable</a> or use replication to get the new data in real time on the OLAP cluster.
In the worst case, if you really need to collocate both, set MR to use less Map and Reduce slots than you&#8217;d normally configure, possibly just one.</p>
</div>
<div class="paragraph">
<p>When HBase is used for OLAP operations, it&#8217;s preferable to set it up in a hardened way like configuring the ZooKeeper session timeout higher and giving more memory to the MemStores (the argument being that the Block Cache won&#8217;t be used much since the workloads are usually long scans).</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="perf.casestudy"><a class="anchor" href="#perf.casestudy"></a>131. Case Studies</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For Performance and Troubleshooting Case Studies, see <a href="#casestudies">Apache HBase Case Studies</a>.</p>
</div>
</div>
</div>
<h1 id="profiler" class="sect0"><a class="anchor" href="#profiler"></a>Profiler Servlet</h1>
<div class="sect1">
<h2 id="_background_2"><a class="anchor" href="#_background_2"></a>132. Background</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBASE-21926 introduced a new servlet that supports integrated profiling via async-profiler.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_prerequisites_2"><a class="anchor" href="#_prerequisites_2"></a>133. Prerequisites</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Go to <a href="https://github.com/jvm-profiling-tools/async-profiler" class="bare">https://github.com/jvm-profiling-tools/async-profiler</a>, download a release appropriate for your platform, and install on every cluster host.</p>
</div>
<div class="paragraph">
<p>Set <code>ASYNC_PROFILER_HOME</code> in the environment (put it in hbase-env.sh) to the root directory of the async-profiler install location, or pass it on the HBase daemon&#8217;s command line as a system property as <code>-Dasync.profiler.home=/path/to/async-profiler</code>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_usage"><a class="anchor" href="#_usage"></a>134. Usage</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Once the prerequisites are satisfied, access to async-profiler is available by way of the HBase UI or direct interaction with the infoserver.</p>
</div>
<div class="paragraph">
<p>Examples:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>To collect 30 second CPU profile of current process (returns FlameGraph svg)
<code>curl <a href="http://localhost:16030/prof" class="bare">http://localhost:16030/prof</a></code></p>
</li>
<li>
<p>To collect 1 minute CPU profile of current process and output in tree format (html)
<code>curl <a href="http://localhost:16030/prof?output=tree&amp;duration=60" class="bare">http://localhost:16030/prof?output=tree&amp;duration=60</a></code></p>
</li>
<li>
<p>To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
<code>curl <a href="http://localhost:16030/prof?event=alloc" class="bare">http://localhost:16030/prof?event=alloc</a></code></p>
</li>
<li>
<p>To collect lock contention profile of current process (returns FlameGraph svg)
<code>curl <a href="http://localhost:16030/prof?event=lock" class="bare">http://localhost:16030/prof?event=lock</a></code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following event types are supported by async-profiler. Use the 'event' parameter to specify. Default is 'cpu'. Not all operating systems will support all types.</p>
</div>
<div class="paragraph">
<p>Perf events:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>cpu</p>
</li>
<li>
<p>page-faults</p>
</li>
<li>
<p>context-switches</p>
</li>
<li>
<p>cycles</p>
</li>
<li>
<p>instructions</p>
</li>
<li>
<p>cache-references</p>
</li>
<li>
<p>cache-misses</p>
</li>
<li>
<p>branches</p>
</li>
<li>
<p>branch-misses</p>
</li>
<li>
<p>bus-cycles</p>
</li>
<li>
<p>L1-dcache-load-misses</p>
</li>
<li>
<p>LLC-load-misses</p>
</li>
<li>
<p>dTLB-load-misses</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Java events:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>alloc</p>
</li>
<li>
<p>lock</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following output formats are supported. Use the 'output' parameter to specify. Default is 'flamegraph'.</p>
</div>
<div class="paragraph">
<p>Output formats:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>summary: A dump of basic profiling statistics.</p>
</li>
<li>
<p>traces: Call traces.</p>
</li>
<li>
<p>flat: Flat profile (top N hot methods).</p>
</li>
<li>
<p>collapsed: Collapsed call traces in the format used by FlameGraph script. This is a collection of call stacks, where each line is a semicolon separated list of frames followed by a counter.</p>
</li>
<li>
<p>svg: FlameGraph in SVG format.</p>
</li>
<li>
<p>tree: Call tree in HTML format.</p>
</li>
<li>
<p>jfr: Call traces in Java Flight Recorder format.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The 'duration' parameter specifies how long to collect trace data before generating output, specified in seconds. The default is 10 seconds.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_ui"><a class="anchor" href="#_ui"></a>135. UI</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In the UI, there is a new entry 'Profiler' in the top menu that will run the default action, which is to profile the CPU usage of the local process for thirty seconds and then produce FlameGraph SVG output.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_notes"><a class="anchor" href="#_notes"></a>136. Notes</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The query parameter <code>pid</code> can be used to specify the process id of a specific process to be profiled. If this parameter is missing the local process in which the infoserver is embedded will be profiled. Profile targets that are not JVMs might work but is not specifically supported. There are security implications. Access to the infoserver should be appropriately restricted.</p>
</div>
</div>
</div>
<h1 id="trouble" class="sect0"><a class="anchor" href="#trouble"></a>Troubleshooting and Debugging Apache HBase</h1>
<div class="sect1">
<h2 id="trouble.general"><a class="anchor" href="#trouble.general"></a>137. General Guidelines</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Always start with the master log (TODO: Which lines?). Normally it&#8217;s just printing the same lines over and over again.
If not, then there&#8217;s an issue.
Google or <a href="http://search-hadoop.com">search-hadoop.com</a> should return some hits for those exceptions you&#8217;re seeing.</p>
</div>
<div class="paragraph">
<p>An error rarely comes alone in Apache HBase, usually when something gets screwed up what will follow may be hundreds of exceptions and stack traces coming from all over the place.
The best way to approach this type of problem is to walk the log up to where it all began, for example one trick with RegionServers is that they will print some metrics when aborting so grepping for <em>Dump</em> should get you around the start of the problem.</p>
</div>
<div class="paragraph">
<p>RegionServer suicides are 'normal', as this is what they do when something goes wrong.
For example, if ulimit and max transfer threads (the two most important initial settings, see <a href="#ulimit">[ulimit]</a> and <a href="#dfs.datanode.max.transfer.threads"><code>dfs.datanode.max.transfer.threads</code> </a>) aren&#8217;t changed, it will make it impossible at some point for DataNodes to create new threads that from the HBase point of view is seen as if HDFS was gone.
Think about what would happen if your MySQL database was suddenly unable to access files on your local file system, well it&#8217;s the same with HBase and HDFS.
Another very common reason to see RegionServers committing seppuku is when they enter prolonged garbage collection pauses that last longer than the default ZooKeeper session timeout.
For more information on GC pauses, see the <a href="https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/">3 part blog post</a> by Todd Lipcon and <a href="#gcpause">Long GC pauses</a> above.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.log"><a class="anchor" href="#trouble.log"></a>138. Logs</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The key process logs are as follows&#8230;&#8203; (replace &lt;user&gt; with the user that started the service, and &lt;hostname&gt; for the machine name)</p>
</div>
<div class="paragraph">
<p>NameNode: <em>$HADOOP_HOME/logs/hadoop-&lt;user&gt;-namenode-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>DataNode: <em>$HADOOP_HOME/logs/hadoop-&lt;user&gt;-datanode-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>JobTracker: <em>$HADOOP_HOME/logs/hadoop-&lt;user&gt;-jobtracker-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>TaskTracker: <em>$HADOOP_HOME/logs/hadoop-&lt;user&gt;-tasktracker-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>HMaster: <em>$HBASE_HOME/logs/hbase-&lt;user&gt;-master-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>RegionServer: <em>$HBASE_HOME/logs/hbase-&lt;user&gt;-regionserver-&lt;hostname&gt;.log</em></p>
</div>
<div class="paragraph">
<p>ZooKeeper: <em>TODO</em></p>
</div>
<div class="sect2">
<h3 id="trouble.log.locations"><a class="anchor" href="#trouble.log.locations"></a>138.1. Log Locations</h3>
<div class="paragraph">
<p>For stand-alone deployments the logs are obviously going to be on a single machine, however this is a development configuration only.
Production deployments need to run on a cluster.</p>
</div>
<div class="sect3">
<h4 id="trouble.log.locations.namenode"><a class="anchor" href="#trouble.log.locations.namenode"></a>138.1.1. NameNode</h4>
<div class="paragraph">
<p>The NameNode log is on the NameNode server.
The HBase Master is typically run on the NameNode server, and well as ZooKeeper.</p>
</div>
<div class="paragraph">
<p>For smaller clusters the JobTracker/ResourceManager is typically run on the NameNode server as well.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.log.locations.datanode"><a class="anchor" href="#trouble.log.locations.datanode"></a>138.1.2. DataNode</h4>
<div class="paragraph">
<p>Each DataNode server will have a DataNode log for HDFS, as well as a RegionServer log for HBase.</p>
</div>
<div class="paragraph">
<p>Additionally, each DataNode server will also have a TaskTracker/NodeManager log for MapReduce task execution.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.log.levels"><a class="anchor" href="#trouble.log.levels"></a>138.2. Log Levels</h3>
<div class="sect3">
<h4 id="rpc.logging"><a class="anchor" href="#rpc.logging"></a>138.2.1. Enabling RPC-level logging</h4>
<div class="paragraph">
<p>Enabling the RPC-level logging on a RegionServer can often give insight on timings at the server.
Once enabled, the amount of log spewed is voluminous.
It is not recommended that you leave this logging on for more than short bursts of time.
To enable RPC-level logging, browse to the RegionServer UI and click on <em>Log Level</em>.
Set the log level to <code>DEBUG</code> for the package <code>org.apache.hadoop.ipc</code> (That&#8217;s right, for <code>hadoop.ipc</code>, NOT, <code>hbase.ipc</code>). Then tail the RegionServers log.
Analyze.</p>
</div>
<div class="paragraph">
<p>To disable, set the logging level back to <code>INFO</code> level.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.log.gc"><a class="anchor" href="#trouble.log.gc"></a>138.3. JVM Garbage Collection Logs</h3>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="paragraph">
<p>All example Garbage Collection logs in this section are based on Java 8 output. The introduction of Unified Logging in Java 9 and newer will result in very different looking logs.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>HBase is memory intensive, and using the default GC you can see long pauses in all threads including the <em>Juliet Pause</em> aka "GC of Death". To help debug this or confirm this is happening GC logging can be turned on in the Java virtual machine.</p>
</div>
<div class="paragraph">
<p>To enable, in <em>hbase-env.sh</em>, uncomment one of the below lines :</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"># This enables basic gc logging to the .out file.
# export SERVER_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps&quot;
# This enables basic gc logging to its own file.
# export SERVER_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:&lt;FILE-PATH&gt;&quot;
# This enables basic GC logging to its own file with automatic log rolling. Only applies to jdk 1.6.0_34+ and 1.7.0_2+.
# export SERVER_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:&lt;FILE-PATH&gt; -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=1 -XX:GCLogFileSize=512M&quot;
# If &lt;FILE-PATH&gt; is not replaced, the log file(.gc) would be generated in the HBASE_LOG_DIR.</code></pre>
</div>
</div>
<div class="paragraph">
<p>At this point you should see logs like so:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="float">64898.952</span>: [GC [<span class="integer">1</span> CMS-initial-mark: <span class="integer">2811538</span>K(<span class="integer">3055704</span>K)] <span class="integer">2812179</span>K(<span class="integer">3061272</span>K), <span class="float">0.0007360</span> secs] [Times: user=<span class="float">0.00</span> sys=<span class="float">0.00</span>, real=<span class="float">0.00</span> secs]
<span class="float">64898.953</span>: [CMS-concurrent-mark-start]
<span class="float">64898.971</span>: [GC <span class="float">64898.971</span>: [ParNew: <span class="integer">5567</span>K-&gt;<span class="integer">576</span>K(<span class="integer">5568</span>K), <span class="float">0.0101110</span> secs] <span class="integer">2817105</span>K-&gt;<span class="integer">2812715</span>K(<span class="integer">3061272</span>K), <span class="float">0.0102200</span> secs] [Times: user=<span class="float">0.07</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]</code></pre>
</div>
</div>
<div class="paragraph">
<p>In this section, the first line indicates a 0.0007360 second pause for the CMS to initially mark.
This pauses the entire VM, all threads for that period of time.</p>
</div>
<div class="paragraph">
<p>The third line indicates a "minor GC", which pauses the VM for 0.0101110 seconds - aka 10 milliseconds.
It has reduced the "ParNew" from about 5.5m to 576k.
Later on in this cycle we see:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="float">64901.445</span>: [CMS-concurrent-mark: <span class="float">1.542</span>/<span class="float">2.492</span> secs] [Times: user=<span class="float">10.49</span> sys=<span class="float">0.33</span>, real=<span class="float">2.49</span> secs]
<span class="float">64901.445</span>: [CMS-concurrent-preclean-start]
<span class="float">64901.453</span>: [GC <span class="float">64901.453</span>: [ParNew: <span class="integer">5505</span>K-&gt;<span class="integer">573</span>K(<span class="integer">5568</span>K), <span class="float">0.0062440</span> secs] <span class="integer">2868746</span>K-&gt;<span class="integer">2864292</span>K(<span class="integer">3061272</span>K), <span class="float">0.0063360</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.476</span>: [GC <span class="float">64901.476</span>: [ParNew: <span class="integer">5563</span>K-&gt;<span class="integer">575</span>K(<span class="integer">5568</span>K), <span class="float">0.0072510</span> secs] <span class="integer">2869283</span>K-&gt;<span class="integer">2864837</span>K(<span class="integer">3061272</span>K), <span class="float">0.0073320</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.01</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.500</span>: [GC <span class="float">64901.500</span>: [ParNew: <span class="integer">5517</span>K-&gt;<span class="integer">573</span>K(<span class="integer">5568</span>K), <span class="float">0.0120390</span> secs] <span class="integer">2869780</span>K-&gt;<span class="integer">2865267</span>K(<span class="integer">3061272</span>K), <span class="float">0.0121150</span> secs] [Times: user=<span class="float">0.09</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.529</span>: [GC <span class="float">64901.529</span>: [ParNew: <span class="integer">5507</span>K-&gt;<span class="integer">569</span>K(<span class="integer">5568</span>K), <span class="float">0.0086240</span> secs] <span class="integer">2870200</span>K-&gt;<span class="integer">2865742</span>K(<span class="integer">3061272</span>K), <span class="float">0.0087180</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.554</span>: [GC <span class="float">64901.555</span>: [ParNew: <span class="integer">5516</span>K-&gt;<span class="integer">575</span>K(<span class="integer">5568</span>K), <span class="float">0.0107130</span> secs] <span class="integer">2870689</span>K-&gt;<span class="integer">2866291</span>K(<span class="integer">3061272</span>K), <span class="float">0.0107820</span> secs] [Times: user=<span class="float">0.06</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.578</span>: [CMS-concurrent-preclean: <span class="float">0.070</span>/<span class="float">0.133</span> secs] [Times: user=<span class="float">0.48</span> sys=<span class="float">0.01</span>, real=<span class="float">0.14</span> secs]
<span class="float">64901.578</span>: [CMS-concurrent-abortable-preclean-start]
<span class="float">64901.584</span>: [GC <span class="float">64901.584</span>: [ParNew: <span class="integer">5504</span>K-&gt;<span class="integer">571</span>K(<span class="integer">5568</span>K), <span class="float">0.0087270</span> secs] <span class="integer">2871220</span>K-&gt;<span class="integer">2866830</span>K(<span class="integer">3061272</span>K), <span class="float">0.0088220</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.609</span>: [GC <span class="float">64901.609</span>: [ParNew: <span class="integer">5512</span>K-&gt;<span class="integer">569</span>K(<span class="integer">5568</span>K), <span class="float">0.0063370</span> secs] <span class="integer">2871771</span>K-&gt;<span class="integer">2867322</span>K(<span class="integer">3061272</span>K), <span class="float">0.0064230</span> secs] [Times: user=<span class="float">0.06</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.615</span>: [CMS-concurrent-abortable-preclean: <span class="float">0.007</span>/<span class="float">0.037</span> secs] [Times: user=<span class="float">0.13</span> sys=<span class="float">0.00</span>, real=<span class="float">0.03</span> secs]
<span class="float">64901.616</span>: [GC[YG occupancy: <span class="integer">645</span> K (<span class="integer">5568</span> K)]<span class="float">64901.616</span>: [Rescan (parallel) , <span class="float">0.0020210</span> secs]<span class="float">64901.618</span>: [weak refs processing, <span class="float">0.0027950</span> secs] [<span class="integer">1</span> CMS-remark: <span class="integer">2866753</span>K(<span class="integer">3055704</span>K)] <span class="integer">2867399</span>K(<span class="integer">3061272</span>K), <span class="float">0.0049380</span> secs] [Times: user=<span class="float">0.00</span> sys=<span class="float">0.01</span>, real=<span class="float">0.01</span> secs]
<span class="float">64901.621</span>: [CMS-concurrent-sweep-start]</code></pre>
</div>
</div>
<div class="paragraph">
<p>The first line indicates that the CMS concurrent mark (finding garbage) has taken 2.4 seconds.
But this is a <em>concurrent</em> 2.4 seconds, Java has not been paused at any point in time.</p>
</div>
<div class="paragraph">
<p>There are a few more minor GCs, then there is a pause at the 2nd last line:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="float">64901.616</span>: [GC[YG occupancy: <span class="integer">645</span> K (<span class="integer">5568</span> K)]<span class="float">64901.616</span>: [Rescan (parallel) , <span class="float">0.0020210</span> secs]<span class="float">64901.618</span>: [weak refs processing, <span class="float">0.0027950</span> secs] [<span class="integer">1</span> CMS-remark: <span class="integer">2866753</span>K(<span class="integer">3055704</span>K)] <span class="integer">2867399</span>K(<span class="integer">3061272</span>K), <span class="float">0.0049380</span> secs] [Times: user=<span class="float">0.00</span> sys=<span class="float">0.01</span>, real=<span class="float">0.01</span> secs]</code></pre>
</div>
</div>
<div class="paragraph">
<p>The pause here is 0.0049380 seconds (aka 4.9 milliseconds) to 'remark' the heap.</p>
</div>
<div class="paragraph">
<p>At this point the sweep starts, and you can watch the heap size go down:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="float">64901.637</span>: [GC <span class="float">64901.637</span>: [ParNew: <span class="integer">5501</span>K-&gt;<span class="integer">569</span>K(<span class="integer">5568</span>K), <span class="float">0.0097350</span> secs] <span class="integer">2871958</span>K-&gt;<span class="integer">2867441</span>K(<span class="integer">3061272</span>K), <span class="float">0.0098370</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
... lines removed ...
<span class="float">64904.936</span>: [GC <span class="float">64904.936</span>: [ParNew: <span class="integer">5532</span>K-&gt;<span class="integer">568</span>K(<span class="integer">5568</span>K), <span class="float">0.0070720</span> secs] <span class="integer">1365024</span>K-&gt;<span class="integer">1360689</span>K(<span class="integer">3061272</span>K), <span class="float">0.0071930</span> secs] [Times: user=<span class="float">0.05</span> sys=<span class="float">0.00</span>, real=<span class="float">0.01</span> secs]
<span class="float">64904.953</span>: [CMS-concurrent-sweep: <span class="float">2.030</span>/<span class="float">3.332</span> secs] [Times: user=<span class="float">9.57</span> sys=<span class="float">0.26</span>, real=<span class="float">3.33</span> secs]</code></pre>
</div>
</div>
<div class="paragraph">
<p>At this point, the CMS sweep took 3.332 seconds, and heap went from about ~ 2.8 GB to 1.3 GB (approximate).</p>
</div>
<div class="paragraph">
<p>The key points here is to keep all these pauses low.
CMS pauses are always low, but if your ParNew starts growing, you can see minor GC pauses approach 100ms, exceed 100ms and hit as high at 400ms.</p>
</div>
<div class="paragraph">
<p>This can be due to the size of the ParNew, which should be relatively small.
If your ParNew is very large after running HBase for a while, in one example a ParNew was about 150MB, then you might have to constrain the size of ParNew (The larger it is, the longer the collections take but if it&#8217;s too small, objects are promoted to old gen too quickly). In the below we constrain new gen size to 64m.</p>
</div>
<div class="paragraph">
<p>Add the below line in <em>hbase-env.sh</em>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">export SERVER_GC_OPTS=&quot;$SERVER_GC_OPTS -XX:NewSize=64m -XX:MaxNewSize=64m&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Similarly, to enable GC logging for client processes, uncomment one of the below lines in <em>hbase-env.sh</em>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"># This enables basic gc logging to the .out file.
# export CLIENT_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps&quot;
# This enables basic gc logging to its own file.
# export CLIENT_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:&lt;FILE-PATH&gt;&quot;
# This enables basic GC logging to its own file with automatic log rolling. Only applies to jdk 1.6.0_34+ and 1.7.0_2+.
# export CLIENT_GC_OPTS=&quot;-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:&lt;FILE-PATH&gt; -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=1 -XX:GCLogFileSize=512M&quot;
# If &lt;FILE-PATH&gt; is not replaced, the log file(.gc) would be generated in the HBASE_LOG_DIR .</code></pre>
</div>
</div>
<div class="paragraph">
<p>For more information on GC pauses, see the <a href="https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/">3 part blog post</a> by Todd Lipcon and <a href="#gcpause">Long GC pauses</a> above.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.resources"><a class="anchor" href="#trouble.resources"></a>139. Resources</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.resources.searchhadoop"><a class="anchor" href="#trouble.resources.searchhadoop"></a>139.1. search-hadoop.com</h3>
<div class="paragraph">
<p><a href="http://search-hadoop.com">search-hadoop.com</a> indexes all the mailing lists and is great for historical searches.
Search here first when you have an issue as its more than likely someone has already had your problem.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.resources.lists"><a class="anchor" href="#trouble.resources.lists"></a>139.2. Mailing Lists</h3>
<div class="paragraph">
<p>Ask a question on the <a href="https://hbase.apache.org/mail-lists.html">Apache HBase mailing lists</a>.
The 'dev' mailing list is aimed at the community of developers actually building Apache HBase and for features currently under development, and 'user' is generally used for questions on released versions of Apache HBase.
Before going to the mailing list, make sure your question has not already been answered by searching the mailing list archives first.
Use <a href="#trouble.resources.searchhadoop">search-hadoop.com</a>.
Take some time crafting your question.
See <a href="http://www.mikeash.com/getting_answers.html">Getting Answers</a> for ideas on crafting good questions.
A quality question that includes all context and exhibits evidence the author has tried to find answers in the manual and out on lists is more likely to get a prompt response.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.resources.slack"><a class="anchor" href="#trouble.resources.slack"></a>139.3. Slack</h3>
<div class="paragraph">
<p>See <a href="http://apache-hbase.slack.com" class="bare">http://apache-hbase.slack.com</a> Channel on Slack</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.resources.irc"><a class="anchor" href="#trouble.resources.irc"></a>139.4. IRC</h3>
<div class="paragraph">
<p>(You will probably get a more prompt response on the Slack channel)</p>
</div>
<div class="paragraph">
<p>#hbase on irc.freenode.net</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.resources.jira"><a class="anchor" href="#trouble.resources.jira"></a>139.5. JIRA</h3>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE">JIRA</a> is also really helpful when looking for Hadoop/HBase-specific issues.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.tools"><a class="anchor" href="#trouble.tools"></a>140. Tools</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.tools.builtin"><a class="anchor" href="#trouble.tools.builtin"></a>140.1. Builtin Tools</h3>
<div class="sect3">
<h4 id="trouble.tools.builtin.webmaster"><a class="anchor" href="#trouble.tools.builtin.webmaster"></a>140.1.1. Master Web Interface</h4>
<div class="paragraph">
<p>The Master starts a web-interface on port 16010 by default.</p>
</div>
<div class="paragraph">
<p>The Master web UI lists created tables and their definition (e.g., ColumnFamilies, blocksize, etc.). Additionally, the available RegionServers in the cluster are listed along with selected high-level metrics (requests, number of regions, usedHeap, maxHeap). The Master web UI allows navigation to each RegionServer&#8217;s web UI.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.builtin.webregion"><a class="anchor" href="#trouble.tools.builtin.webregion"></a>140.1.2. RegionServer Web Interface</h4>
<div class="paragraph">
<p>RegionServers starts a web-interface on port 16030 by default.</p>
</div>
<div class="paragraph">
<p>The RegionServer web UI lists online regions and their start/end keys, as well as point-in-time RegionServer metrics (requests, regions, storeFileIndexSize, compactionQueueSize, etc.).</p>
</div>
<div class="paragraph">
<p>See <a href="#hbase_metrics">HBase Metrics</a> for more information in metric definitions.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.builtin.zkcli"><a class="anchor" href="#trouble.tools.builtin.zkcli"></a>140.1.3. zkcli</h4>
<div class="paragraph">
<p><code>zkcli</code> is a very useful tool for investigating ZooKeeper-related issues.
To invoke:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">./hbase zkcli -server host:port &lt;cmd&gt; &lt;args&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>The commands (and arguments) are:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> connect host:port
get path [watch]
ls path [watch]
set path data [version]
delquota [-n|-b] path
quit
printwatches on|off
create [-s] [-e] path data acl
stat path [watch]
close
ls2 path [watch]
history
listquota path
setAcl path acl
getAcl path
sync path
redo cmdno
addauth scheme auth
delete path [version]
setquota -n|-b val path</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.maintenancemode"><a class="anchor" href="#trouble.tools.maintenancemode"></a>140.1.4. Maintenance Mode</h4>
<div class="paragraph">
<p>If the cluster has gotten stuck in some state and the standard techniques aren&#8217;t making progress,
it is possible to restart the cluster in "maintenance mode." This mode features drastically
reduced capabilities and surface area, making it easier to enact very low-level changes such
as repairing/recovering the <code>hbase:meta</code> table.</p>
</div>
<div class="paragraph">
<p>To enter maintenance mode, set <code>hbase.master.maintenance_mode</code> to <code>true</code> either in your
<code>hbase-site.xml</code> or via system propery when starting the master process (<code>-D&#8230;&#8203;=true</code>). Entering
and exiting this mode requires a service restart, however the typical use will be when HBase Master
is already facing startup difficulties.</p>
</div>
<div class="paragraph">
<p>When maintenance mode is enabled, the master will host all system tables - ensure that it has
enough memory to do so. RegionServers will not be assigned any regions from user-space tables;
in fact, they will go completely unused while in maintenance mode. Additionally, the master will
not load any coprocessors, will not run any normalization or merge/split operations, and will not
enforce quotas.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.tools.external"><a class="anchor" href="#trouble.tools.external"></a>140.2. External Tools</h3>
<div class="sect3">
<h4 id="trouble.tools.tail"><a class="anchor" href="#trouble.tools.tail"></a>140.2.1. tail</h4>
<div class="paragraph">
<p><code>tail</code> is the command line tool that lets you look at the end of a file.
Add the <code>-f</code> option and it will refresh when new data is available.
It&#8217;s useful when you are wondering what&#8217;s happening, for example, when a cluster is taking a long time to shutdown or startup as you can just fire a new terminal and tail the master log (and maybe a few RegionServers).</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.top"><a class="anchor" href="#trouble.tools.top"></a>140.2.2. top</h4>
<div class="paragraph">
<p><code>top</code> is probably one of the most important tools when first trying to see what&#8217;s running on a machine and how the resources are consumed.
Here&#8217;s an example from production system:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">top - <span class="integer">14</span>:<span class="integer">46</span>:<span class="integer">59</span> up <span class="integer">39</span> days, <span class="integer">11</span>:<span class="integer">55</span>, <span class="integer">1</span> user, load average: <span class="float">3.75</span>, <span class="float">3.57</span>, <span class="float">3.84</span>
Tasks: <span class="integer">309</span> total, <span class="integer">1</span> running, <span class="integer">308</span> sleeping, <span class="integer">0</span> stopped, <span class="integer">0</span> zombie
Cpu(s): <span class="float">4.5</span>%us, <span class="float">1.6</span>%sy, <span class="float">0.0</span>%ni, <span class="float">91.7</span>%id, <span class="float">1.4</span>%wa, <span class="float">0.1</span>%hi, <span class="float">0.6</span>%si, <span class="float">0.0</span>%st
Mem: <span class="integer">24414432</span>k total, <span class="integer">24296956</span>k used, <span class="integer">117476</span>k free, <span class="integer">7196</span>k buffers
Swap: <span class="integer">16008732</span>k total, <span class="integer">14348</span>k used, <span class="integer">15994384</span>k free, <span class="integer">11106908</span>k cached
PID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND
<span class="integer">15558</span> hadoop <span class="integer">18</span> -<span class="integer">2</span> <span class="integer">3292</span>m <span class="float">2.4</span>g <span class="integer">3556</span> S <span class="integer">79</span> <span class="float">10.4</span> <span class="integer">6523</span>:<span class="integer">52</span> java
<span class="integer">13268</span> hadoop <span class="integer">18</span> -<span class="integer">2</span> <span class="integer">8967</span>m <span class="float">8.2</span>g <span class="integer">4104</span> S <span class="integer">21</span> <span class="float">35.1</span> <span class="integer">5170</span>:<span class="integer">30</span> java
<span class="integer">8895</span> hadoop <span class="integer">18</span> -<span class="integer">2</span> <span class="integer">1581</span>m <span class="integer">497</span>m <span class="integer">3420</span> S <span class="integer">11</span> <span class="float">2.1</span> <span class="integer">4002</span>:<span class="integer">32</span> java
<span class="error"></span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Here we can see that the system load average during the last five minutes is 3.75, which very roughly means that on average 3.75 threads were waiting for CPU time during these 5 minutes.
In general, the <em>perfect</em> utilization equals to the number of cores, under that number the machine is under utilized and over that the machine is over utilized.
This is an important concept, see this article to understand it more: <a href="http://www.linuxjournal.com/article/9001" class="bare">http://www.linuxjournal.com/article/9001</a>.</p>
</div>
<div class="paragraph">
<p>Apart from load, we can see that the system is using almost all its available RAM but most of it is used for the OS cache (which is good). The swap only has a few KBs in it and this is wanted, high numbers would indicate swapping activity which is the nemesis of performance of Java systems.
Another way to detect swapping is when the load average goes through the roof (although this could also be caused by things like a dying disk, among others).</p>
</div>
<div class="paragraph">
<p>The list of processes isn&#8217;t super useful by default, all we know is that 3 java processes are using about 111% of the CPUs.
To know which is which, simply type <code>c</code> and each line will be expanded.
Typing <code>1</code> will give you the detail of how each CPU is used instead of the average for all of them like shown here.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.jps"><a class="anchor" href="#trouble.tools.jps"></a>140.2.3. jps</h4>
<div class="paragraph">
<p><code>jps</code> is shipped with every JDK and gives the java process ids for the current user (if root, then it gives the ids for all users). Example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">hadoop@sv4borg12:~$ jps
1322 TaskTracker
17789 HRegionServer
27862 Child
1158 DataNode
25115 HQuorumPeer
2950 Jps
19750 ThriftServer
18776 jmx</code></pre>
</div>
</div>
<div class="paragraph">
<p>In order, we see a:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Hadoop TaskTracker, manages the local Childs</p>
</li>
<li>
<p>HBase RegionServer, serves regions</p>
</li>
<li>
<p>Child, its MapReduce task, cannot tell which type exactly</p>
</li>
<li>
<p>Hadoop TaskTracker, manages the local Childs</p>
</li>
<li>
<p>Hadoop DataNode, serves blocks</p>
</li>
<li>
<p>HQuorumPeer, a ZooKeeper ensemble member</p>
</li>
<li>
<p>Jps, well&#8230;&#8203; it&#8217;s the current process</p>
</li>
<li>
<p>ThriftServer, it&#8217;s a special one will be running only if thrift was started</p>
</li>
<li>
<p>jmx, this is a local process that&#8217;s part of our monitoring platform ( poorly named maybe). You probably don&#8217;t have that.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>You can then do stuff like checking out the full command line that started the process:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">hadoop@sv4borg12:~$ ps aux | grep HRegionServer
hadoop 17789 155 35.2 9067824 8604364 ? S&amp;lt;l Mar04 9855:48 /usr/java/jdk1.6.0_14/bin/java -Xmx8000m -XX:+DoEscapeAnalysis -XX:+AggressiveOpts -XX:+UseConcMarkSweepGC -XX:NewSize=64m -XX:MaxNewSize=64m -XX:CMSInitiatingOccupancyFraction=88 -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -Xloggc:/export1/hadoop/logs/gc-hbase.log -Dcom.sun.management.jmxremote.port=10102 -Dcom.sun.management.jmxremote.authenticate=true -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.password.file=/home/hadoop/hbase/conf/jmxremote.password -Dcom.sun.management.jmxremote -Dhbase.log.dir=/export1/hadoop/logs -Dhbase.log.file=hbase-hadoop-regionserver-sv4borg12.log -Dhbase.home.dir=/home/hadoop/hbase -Dhbase.id.str=hadoop -Dhbase.root.logger=INFO,DRFA -Djava.library.path=/home/hadoop/hbase/lib/native/Linux-amd64-64 -classpath /home/hadoop/hbase/bin/../conf:[many jars]:/home/hadoop/hadoop/conf org.apache.hadoop.hbase.regionserver.HRegionServer start</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.jstack"><a class="anchor" href="#trouble.tools.jstack"></a>140.2.4. jstack</h4>
<div class="paragraph">
<p><code>jstack</code> is one of the most important tools when trying to figure out what a java process is doing apart from looking at the logs.
It has to be used in conjunction with jps in order to give it a process id.
It shows a list of threads, each one has a name, and they appear in the order that they were created (so the top ones are the most recent threads). Here are a few example:</p>
</div>
<div class="paragraph">
<p>The main thread of a RegionServer waiting for something to do from the master:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">regionserver60020</span><span class="delimiter">&quot;</span></span> prio=<span class="integer">10</span> tid=<span class="hex">0x0000000040ab4000</span> nid=<span class="hex">0x45cf</span> waiting on condition [<span class="hex">0x00007f16b6a96000</span>.<span class="float">.0</span>x00007f16b6a96a70]
java.lang.Thread.State: TIMED_WAITING (parking)
at sun.misc.Unsafe.park(Native <span class="predefined-type">Method</span>)
- parking to wait <span class="keyword">for</span> &lt;<span class="hex">0x00007f16cd5c2f30</span>&gt; (a java.util.concurrent.locks.AbstractQueuedSynchronizer<span class="error">$</span>ConditionObject)
at java.util.concurrent.locks.LockSupport.parkNanos(<span class="predefined-type">LockSupport</span>.java:<span class="integer">198</span>)
at java.util.concurrent.locks.AbstractQueuedSynchronizer<span class="error">$</span>ConditionObject.awaitNanos(<span class="predefined-type">AbstractQueuedSynchronizer</span>.java:<span class="integer">1963</span>)
at java.util.concurrent.LinkedBlockingQueue.poll(<span class="predefined-type">LinkedBlockingQueue</span>.java:<span class="integer">395</span>)
at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:<span class="integer">647</span>)
at java.lang.Thread.run(<span class="predefined-type">Thread</span>.java:<span class="integer">619</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>The MemStore flusher thread that is currently flushing to a file:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">regionserver60020.cacheFlusher</span><span class="delimiter">&quot;</span></span> daemon prio=<span class="integer">10</span> tid=<span class="hex">0x0000000040f4e000</span> nid=<span class="hex">0x45eb</span> in <span class="predefined-type">Object</span>.wait() [<span class="hex">0x00007f16b5b86000</span>.<span class="float">.0</span>x00007f16b5b87af0]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native <span class="predefined-type">Method</span>)
at java.lang.Object.wait(<span class="predefined-type">Object</span>.java:<span class="integer">485</span>)
at org.apache.hadoop.ipc.Client.call(Client.java:<span class="integer">803</span>)
- locked &lt;<span class="hex">0x00007f16cb14b3a8</span>&gt; (a org.apache.hadoop.ipc.Client<span class="error">$</span>Call)
at org.apache.hadoop.ipc.RPC<span class="error">$</span>Invoker.invoke(RPC.java:<span class="integer">221</span>)
at <span class="error">$</span>Proxy1.complete(Unknown <span class="predefined-type">Source</span>)
at sun.reflect.GeneratedMethodAccessor38.invoke(Unknown <span class="predefined-type">Source</span>)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:<span class="integer">25</span>)
at java.lang.reflect.Method.invoke(<span class="predefined-type">Method</span>.java:<span class="integer">597</span>)
at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:<span class="integer">82</span>)
at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:<span class="integer">59</span>)
at <span class="error">$</span>Proxy1.complete(Unknown <span class="predefined-type">Source</span>)
at org.apache.hadoop.hdfs.DFSClient<span class="error">$</span>DFSOutputStream.closeInternal(DFSClient.java:<span class="integer">3390</span>)
- locked &lt;<span class="hex">0x00007f16cb14b470</span>&gt; (a org.apache.hadoop.hdfs.DFSClient<span class="error">$</span>DFSOutputStream)
at org.apache.hadoop.hdfs.DFSClient<span class="error">$</span>DFSOutputStream.close(DFSClient.java:<span class="integer">3304</span>)
at org.apache.hadoop.fs.FSDataOutputStream<span class="error">$</span>PositionCache.close(FSDataOutputStream.java:<span class="integer">61</span>)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:<span class="integer">86</span>)
at org.apache.hadoop.hbase.io.hfile.HFile<span class="error">$</span><span class="predefined-type">Writer</span>.close(HFile.java:<span class="integer">650</span>)
at org.apache.hadoop.hbase.regionserver.StoreFile<span class="error">$</span><span class="predefined-type">Writer</span>.close(StoreFile.java:<span class="integer">853</span>)
at org.apache.hadoop.hbase.regionserver.Store.internalFlushCache(Store.java:<span class="integer">467</span>)
- locked &lt;<span class="hex">0x00007f16d00e6f08</span>&gt; (a java.lang.Object)
at org.apache.hadoop.hbase.regionserver.Store.flushCache(Store.java:<span class="integer">427</span>)
at org.apache.hadoop.hbase.regionserver.Store.access<span class="error">$</span><span class="integer">100</span>(Store.java:<span class="integer">80</span>)
at org.apache.hadoop.hbase.regionserver.Store<span class="error">$</span>StoreFlusherImpl.flushCache(Store.java:<span class="integer">1359</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:<span class="integer">907</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:<span class="integer">834</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:<span class="integer">786</span>)
at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:<span class="integer">250</span>)
at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:<span class="integer">224</span>)
at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.java:<span class="integer">146</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>A handler thread that&#8217;s waiting for stuff to do (like put, delete, scan, etc.):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">IPC Server handler 16 on 60020</span><span class="delimiter">&quot;</span></span> daemon prio=<span class="integer">10</span> tid=<span class="hex">0x00007f16b011d800</span> nid=<span class="hex">0x4a5e</span> waiting on condition [<span class="hex">0x00007f16afefd000</span>.<span class="float">.0</span>x00007f16afefd9f0]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native <span class="predefined-type">Method</span>)
- parking to wait <span class="keyword">for</span> &lt;<span class="hex">0x00007f16cd3f8dd8</span>&gt; (a java.util.concurrent.locks.AbstractQueuedSynchronizer<span class="error">$</span>ConditionObject)
at java.util.concurrent.locks.LockSupport.park(<span class="predefined-type">LockSupport</span>.java:<span class="integer">158</span>)
at java.util.concurrent.locks.AbstractQueuedSynchronizer<span class="error">$</span>ConditionObject.await(<span class="predefined-type">AbstractQueuedSynchronizer</span>.java:<span class="integer">1925</span>)
at java.util.concurrent.LinkedBlockingQueue.take(<span class="predefined-type">LinkedBlockingQueue</span>.java:<span class="integer">358</span>)
at org.apache.hadoop.hbase.ipc.HBaseServer<span class="error">$</span><span class="predefined-type">Handler</span>.run(HBaseServer.java:<span class="integer">1013</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>And one that&#8217;s busy doing an increment of a counter (it&#8217;s in the phase where it&#8217;s trying to create a scanner in order to read the last value):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">IPC Server handler 66 on 60020</span><span class="delimiter">&quot;</span></span> daemon prio=<span class="integer">10</span> tid=<span class="hex">0x00007f16b006e800</span> nid=<span class="hex">0x4a90</span> runnable [<span class="hex">0x00007f16acb77000</span>.<span class="float">.0</span>x00007f16acb77cf0]
java.lang.Thread.State: RUNNABLE
at org.apache.hadoop.hbase.regionserver.KeyValueHeap.&lt;init&gt;(KeyValueHeap.java:<span class="integer">56</span>)
at org.apache.hadoop.hbase.regionserver.StoreScanner.&lt;init&gt;(StoreScanner.java:<span class="integer">79</span>)
at org.apache.hadoop.hbase.regionserver.Store.getScanner(Store.java:<span class="integer">1202</span>)
at org.apache.hadoop.hbase.regionserver.HRegion<span class="error">$</span>RegionScanner.&lt;init&gt;(HRegion.java:<span class="integer">2209</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.instantiateInternalScanner(HRegion.java:<span class="integer">1063</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:<span class="integer">1055</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:<span class="integer">1039</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.getLastIncrement(HRegion.java:<span class="integer">2875</span>)
at org.apache.hadoop.hbase.regionserver.HRegion.incrementColumnValue(HRegion.java:<span class="integer">2978</span>)
at org.apache.hadoop.hbase.regionserver.HRegionServer.incrementColumnValue(HRegionServer.java:<span class="integer">2433</span>)
at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown <span class="predefined-type">Source</span>)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:<span class="integer">25</span>)
at java.lang.reflect.Method.invoke(<span class="predefined-type">Method</span>.java:<span class="integer">597</span>)
at org.apache.hadoop.hbase.ipc.HBaseRPC<span class="error">$</span>Server.call(HBaseRPC.java:<span class="integer">560</span>)
at org.apache.hadoop.hbase.ipc.HBaseServer<span class="error">$</span><span class="predefined-type">Handler</span>.run(HBaseServer.java:<span class="integer">1027</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>A thread that receives data from HDFS:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">IPC Client (47) connection to sv4borg9/10.4.24.40:9000 from hadoop</span><span class="delimiter">&quot;</span></span> daemon prio=<span class="integer">10</span> tid=<span class="hex">0x00007f16a02d0000</span> nid=<span class="hex">0x4fa3</span> runnable [<span class="hex">0x00007f16b517d000</span>.<span class="float">.0</span>x00007f16b517dbf0]
java.lang.Thread.State: RUNNABLE
at sun.nio.ch.EPollArrayWrapper.epollWait(Native <span class="predefined-type">Method</span>)
at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:<span class="integer">215</span>)
at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:<span class="integer">65</span>)
at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:<span class="integer">69</span>)
- locked &lt;<span class="hex">0x00007f17d5b68c00</span>&gt; (a sun.nio.ch.Util<span class="error">$</span><span class="integer">1</span>)
- locked &lt;<span class="hex">0x00007f17d5b68be8</span>&gt; (a java.util.Collections<span class="error">$</span>UnmodifiableSet)
- locked &lt;<span class="hex">0x00007f1877959b50</span>&gt; (a sun.nio.ch.EPollSelectorImpl)
at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:<span class="integer">80</span>)
at org.apache.hadoop.net.SocketIOWithTimeout<span class="error">$</span>SelectorPool.select(SocketIOWithTimeout.java:<span class="integer">332</span>)
at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:<span class="integer">157</span>)
at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:<span class="integer">155</span>)
at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:<span class="integer">128</span>)
at java.io.FilterInputStream.read(<span class="predefined-type">FilterInputStream</span>.java:<span class="integer">116</span>)
at org.apache.hadoop.ipc.Client<span class="error">$</span><span class="predefined-type">Connection</span><span class="error">$</span>PingInputStream.read(Client.java:<span class="integer">304</span>)
at java.io.BufferedInputStream.fill(<span class="predefined-type">BufferedInputStream</span>.java:<span class="integer">218</span>)
at java.io.BufferedInputStream.read(<span class="predefined-type">BufferedInputStream</span>.java:<span class="integer">237</span>)
- locked &lt;<span class="hex">0x00007f1808539178</span>&gt; (a java.io.BufferedInputStream)
at java.io.DataInputStream.readInt(<span class="predefined-type">DataInputStream</span>.java:<span class="integer">370</span>)
at org.apache.hadoop.ipc.Client<span class="error">$</span><span class="predefined-type">Connection</span>.receiveResponse(Client.java:<span class="integer">569</span>)
at org.apache.hadoop.ipc.Client<span class="error">$</span><span class="predefined-type">Connection</span>.run(Client.java:<span class="integer">477</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>And here is a master trying to recover a lease after a RegionServer died:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="string"><span class="delimiter">&quot;</span><span class="content">LeaseChecker</span><span class="delimiter">&quot;</span></span> daemon prio=<span class="integer">10</span> tid=<span class="hex">0x00000000407ef800</span> nid=<span class="hex">0x76cd</span> waiting on condition [<span class="hex">0x00007f6d0eae2000</span>.<span class="float">.0</span>x00007f6d0eae2a70]
--
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native <span class="predefined-type">Method</span>)
at java.lang.Object.wait(<span class="predefined-type">Object</span>.java:<span class="integer">485</span>)
at org.apache.hadoop.ipc.Client.call(Client.java:<span class="integer">726</span>)
- locked &lt;<span class="hex">0x00007f6d1cd28f80</span>&gt; (a org.apache.hadoop.ipc.Client<span class="error">$</span>Call)
at org.apache.hadoop.ipc.RPC<span class="error">$</span>Invoker.invoke(RPC.java:<span class="integer">220</span>)
at <span class="error">$</span>Proxy1.recoverBlock(Unknown <span class="predefined-type">Source</span>)
at org.apache.hadoop.hdfs.DFSClient<span class="error">$</span>DFSOutputStream.processDatanodeError(DFSClient.java:<span class="integer">2636</span>)
at org.apache.hadoop.hdfs.DFSClient<span class="error">$</span>DFSOutputStream.&lt;init&gt;(DFSClient.java:<span class="integer">2832</span>)
at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:<span class="integer">529</span>)
at org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:<span class="integer">186</span>)
at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:<span class="integer">530</span>)
at org.apache.hadoop.hbase.util.FSUtils.recoverFileLease(FSUtils.java:<span class="integer">619</span>)
at org.apache.hadoop.hbase.regionserver.wal.HLog.splitLog(HLog.java:<span class="integer">1322</span>)
at org.apache.hadoop.hbase.regionserver.wal.HLog.splitLog(HLog.java:<span class="integer">1210</span>)
at org.apache.hadoop.hbase.master.HMaster.splitLogAfterStartup(HMaster.java:<span class="integer">648</span>)
at org.apache.hadoop.hbase.master.HMaster.joinCluster(HMaster.java:<span class="integer">572</span>)
at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:<span class="integer">503</span>)</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.opentsdb"><a class="anchor" href="#trouble.tools.opentsdb"></a>140.2.5. OpenTSDB</h4>
<div class="paragraph">
<p><a href="http://opentsdb.net">OpenTSDB</a> is an excellent alternative to Ganglia as it uses Apache HBase to store all the time series and doesn&#8217;t have to downsample.
Monitoring your own HBase cluster that hosts OpenTSDB is a good exercise.</p>
</div>
<div class="paragraph">
<p>Here&#8217;s an example of a cluster that&#8217;s suffering from hundreds of compactions launched almost all around the same time, which severely affects the IO performance: (TODO: insert graph plotting compactionQueueSize)</p>
</div>
<div class="paragraph">
<p>It&#8217;s a good practice to build dashboards with all the important graphs per machine and per cluster so that debugging issues can be done with a single quick look.
For example, at StumbleUpon there&#8217;s one dashboard per cluster with the most important metrics from both the OS and Apache HBase.
You can then go down at the machine level and get even more detailed metrics.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.tools.clustersshtop"><a class="anchor" href="#trouble.tools.clustersshtop"></a>140.2.6. clusterssh+top</h4>
<div class="paragraph">
<p>clusterssh+top, it&#8217;s like a poor man&#8217;s monitoring system and it can be quite useful when you have only a few machines as it&#8217;s very easy to setup.
Starting clusterssh will give you one terminal per machine and another terminal in which whatever you type will be retyped in every window.
This means that you can type <code>top</code> once and it will start it for all of your machines at the same time giving you full view of the current state of your cluster.
You can also tail all the logs at the same time, edit files, etc.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.client"><a class="anchor" href="#trouble.client"></a>141. Client</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For more information on the HBase client, see <a href="#architecture.client">client</a>.</p>
</div>
<div class="sect2">
<h3 id="trouble.client.scantimeout"><a class="anchor" href="#trouble.client.scantimeout"></a>141.1. ScannerTimeoutException or UnknownScannerException</h3>
<div class="paragraph">
<p>This is thrown if the time between RPC calls from the client to RegionServer exceeds the scan timeout.
For example, if <code>Scan.setCaching</code> is set to 500, then there will be an RPC call to fetch the next batch of rows every 500 <code>.next()</code> calls on the ResultScanner because data is being transferred in blocks of 500 rows to the client.
Reducing the setCaching value may be an option, but setting this value too low makes for inefficient processing on numbers of rows.</p>
</div>
<div class="paragraph">
<p>See <a href="#perf.hbase.client.caching">Scan Caching</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_performance_differences_in_thrift_and_java_apis"><a class="anchor" href="#_performance_differences_in_thrift_and_java_apis"></a>141.2. Performance Differences in Thrift and Java APIs</h3>
<div class="paragraph">
<p>Poor performance, or even <code>ScannerTimeoutExceptions</code>, can occur if <code>Scan.setCaching</code> is too high, as discussed in <a href="#trouble.client.scantimeout">ScannerTimeoutException or UnknownScannerException</a>.
If the Thrift client uses the wrong caching settings for a given workload, performance can suffer compared to the Java API.
To set caching for a given scan in the Thrift client, use the <code>scannerGetList(scannerId, numRows)</code> method, where <code>numRows</code> is an integer representing the number of rows to cache.
In one case, it was found that reducing the cache for Thrift scans from 1000 to 100 increased performance to near parity with the Java API given the same queries.</p>
</div>
<div class="paragraph">
<p>See also Jesse Andersen&#8217;s <a href="http://blog.cloudera.com/blog/2014/04/how-to-use-the-hbase-thrift-interface-part-3-using-scans/">blog post</a> about using Scans with Thrift.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.lease.exception"><a class="anchor" href="#trouble.client.lease.exception"></a>141.3. <code>LeaseException</code> when calling <code>Scanner.next</code></h3>
<div class="paragraph">
<p>In some situations clients that fetch data from a RegionServer get a LeaseException instead of the usual <a href="#trouble.client.scantimeout">ScannerTimeoutException or UnknownScannerException</a>.
Usually the source of the exception is <code>org.apache.hadoop.hbase.regionserver.Leases.removeLease(Leases.java:230)</code> (line number may vary). It tends to happen in the context of a slow/freezing <code>RegionServer#next</code> call.
It can be prevented by having <code>hbase.rpc.timeout</code> &gt; <code>hbase.client.scanner.timeout.period</code>.
Harsh J investigated the issue as part of the mailing list thread <a href="https://mail-archives.apache.org/mod_mbox/hbase-user/201209.mbox/%3CCAOcnVr3R-LqtKhFsk8Bhrm-YW2i9O6J6Fhjz2h7q6_sxvwd2yw%40mail.gmail.com%3E">HBase, mail # user - Lease does not exist exceptions</a></p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.scarylogs"><a class="anchor" href="#trouble.client.scarylogs"></a>141.4. Shell or client application throws lots of scary exceptions during normal operation</h3>
<div class="paragraph">
<p>Since 0.20.0 the default log level for `org.apache.hadoop.hbase.*`is DEBUG.</p>
</div>
<div class="paragraph">
<p>On your clients, edit <em>$HBASE_HOME/conf/log4j.properties</em> and change this: <code>log4j.logger.org.apache.hadoop.hbase=DEBUG</code> to this: <code>log4j.logger.org.apache.hadoop.hbase=INFO</code>, or even <code>log4j.logger.org.apache.hadoop.hbase=WARN</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.longpauseswithcompression"><a class="anchor" href="#trouble.client.longpauseswithcompression"></a>141.5. Long Client Pauses With Compression</h3>
<div class="paragraph">
<p>This is a fairly frequent question on the Apache HBase dist-list.
The scenario is that a client is typically inserting a lot of data into a relatively un-optimized HBase cluster.
Compression can exacerbate the pauses, although it is not the source of the problem.</p>
</div>
<div class="paragraph">
<p>See <a href="#precreate.regions">Table Creation: Pre-Creating Regions</a> on the pattern for pre-creating regions and confirm that the table isn&#8217;t starting with a single region.</p>
</div>
<div class="paragraph">
<p>See <a href="#perf.configurations">HBase Configurations</a> for cluster configuration, particularly <code>hbase.hstore.blockingStoreFiles</code>, <code>hbase.hregion.memstore.block.multiplier</code>, <code>MAX_FILESIZE</code> (region size), and <code>MEMSTORE_FLUSHSIZE.</code></p>
</div>
<div class="paragraph">
<p>A slightly longer explanation of why pauses can happen is as follows: Puts are sometimes blocked on the MemStores which are blocked by the flusher thread which is blocked because there are too many files to compact because the compactor is given too many small files to compact and has to compact the same data repeatedly.
This situation can occur even with minor compactions.
Compounding this situation, Apache HBase doesn&#8217;t compress data in memory.
Thus, the 64MB that lives in the MemStore could become a 6MB file after compression - which results in a smaller StoreFile.
The upside is that more data is packed into the same region, but performance is achieved by being able to write larger files - which is why HBase waits until the flushsize before writing a new StoreFile.
And smaller StoreFiles become targets for compaction.
Without compression the files are much bigger and don&#8217;t need as much compaction, however this is at the expense of I/O.</p>
</div>
<div class="paragraph">
<p>For additional information, see this thread on <a href="http://search-hadoop.com/m/WUnLM6ojHm1/Long+client+pauses+with+compression&amp;subj=Long+client+pauses+with+compression">Long client pauses with compression</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.security.rpc.krb"><a class="anchor" href="#trouble.client.security.rpc.krb"></a>141.6. Secure Client Connect ([Caused by GSSException: No valid credentials provided&#8230;&#8203;])</h3>
<div class="paragraph">
<p>You may encounter the following error:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Secure Client Connect ([Caused by GSSException: No valid credentials provided
(Mechanism level: Request is a replay (34) V PROCESS_TGS)])</pre>
</div>
</div>
<div class="paragraph">
<p>This issue is caused by bugs in the MIT Kerberos replay_cache component, <a href="http://krbdev.mit.edu/rt/Ticket/Display.html?id=1201">#1201</a> and <a href="http://krbdev.mit.edu/rt/Ticket/Display.html?id=5924">#5924</a>.
These bugs caused the old version of krb5-server to erroneously block subsequent requests sent from a Principal.
This caused krb5-server to block the connections sent from one Client (one HTable instance with multi-threading connection instances for each RegionServer); Messages, such as <code>Request is a replay (34)</code>, are logged in the client log You can ignore the messages, because HTable will retry 5 * 10 (50) times for each failed connection by default.
HTable will throw IOException if any connection to the RegionServer fails after the retries, so that the user client code for HTable instance can handle it further.
NOTE: <code>HTable</code> is deprecated in HBase 1.0, in favor of <code>Table</code>.</p>
</div>
<div class="paragraph">
<p>Alternatively, update krb5-server to a version which solves these issues, such as krb5-server-1.10.3.
See JIRA <a href="https://issues.apache.org/jira/browse/HBASE-10379">HBASE-10379</a> for more details.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.zookeeper"><a class="anchor" href="#trouble.client.zookeeper"></a>141.7. ZooKeeper Client Connection Errors</h3>
<div class="paragraph">
<p>Errors like this&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">11</span>/<span class="octal">07</span>/<span class="octal">05</span> <span class="integer">11</span>:<span class="integer">26</span>:<span class="integer">41</span> WARN zookeeper.ClientCnxn: Session <span class="hex">0x0</span> <span class="keyword">for</span> server <span class="predefined-constant">null</span>,
unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: <span class="predefined-type">Connection</span> refused: no further information
at sun.nio.ch.SocketChannelImpl.checkConnect(Native <span class="predefined-type">Method</span>)
at sun.nio.ch.SocketChannelImpl.finishConnect(Unknown <span class="predefined-type">Source</span>)
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.run(ClientCnxn.java:<span class="integer">1078</span>)
<span class="integer">11</span>/<span class="octal">07</span>/<span class="octal">05</span> <span class="integer">11</span>:<span class="integer">26</span>:<span class="integer">43</span> INFO zookeeper.ClientCnxn: Opening socket connection to
server localhost/<span class="float">127.0</span><span class="float">.0</span><span class="float">.1</span>:<span class="integer">2181</span>
<span class="integer">11</span>/<span class="octal">07</span>/<span class="octal">05</span> <span class="integer">11</span>:<span class="integer">26</span>:<span class="integer">44</span> WARN zookeeper.ClientCnxn: Session <span class="hex">0x0</span> <span class="keyword">for</span> server <span class="predefined-constant">null</span>,
unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: <span class="predefined-type">Connection</span> refused: no further information
at sun.nio.ch.SocketChannelImpl.checkConnect(Native <span class="predefined-type">Method</span>)
at sun.nio.ch.SocketChannelImpl.finishConnect(Unknown <span class="predefined-type">Source</span>)
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.run(ClientCnxn.java:<span class="integer">1078</span>)
<span class="integer">11</span>/<span class="octal">07</span>/<span class="octal">05</span> <span class="integer">11</span>:<span class="integer">26</span>:<span class="integer">45</span> INFO zookeeper.ClientCnxn: Opening socket connection to
server localhost/<span class="float">127.0</span><span class="float">.0</span><span class="float">.1</span>:<span class="integer">2181</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;are either due to ZooKeeper being down, or unreachable due to network issues.</p>
</div>
<div class="paragraph">
<p>The utility <a href="#trouble.tools.builtin.zkcli">zkcli</a> may help investigate ZooKeeper issues.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.oome.directmemory.leak"><a class="anchor" href="#trouble.client.oome.directmemory.leak"></a>141.8. Client running out of memory though heap size seems to be stable (but the off-heap/direct heap keeps growing)</h3>
<div class="paragraph">
<p>You are likely running into the issue that is described and worked through in the mail thread <a href="http://search-hadoop.com/m/ubhrX8KvcH/Suspected+memory+leak&amp;subj=Re+Suspected+memory+leak">HBase, mail # user - Suspected memory leak</a> and continued over in <a href="http://search-hadoop.com/m/p2Agc1Zy7Va/MaxDirectMemorySize+Was%253A+Suspected+memory+leak&amp;subj=Re+FeedbackRe+Suspected+memory+leak">HBase, mail # dev - FeedbackRe: Suspected memory leak</a>.
A workaround is passing your client-side JVM a reasonable value for <code>-XX:MaxDirectMemorySize</code>.
By default, the <code>MaxDirectMemorySize</code> is equal to your <code>-Xmx</code> max heapsize setting (if <code>-Xmx</code> is set). Try setting it to something smaller (for example, one user had success setting it to <code>1g</code> when they had a client-side heap of <code>12g</code>). If you set it too small, it will bring on <code>FullGCs</code> so keep it a bit hefty.
You want to make this setting client-side only especially if you are running the new experimental server-side off-heap cache since this feature depends on being able to use big direct buffers (You may have to keep separate client-side and server-side config dirs).</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.client.security.rpc"><a class="anchor" href="#trouble.client.security.rpc"></a>141.9. Secure Client Cannot Connect ([Caused by GSSException: No valid credentials provided(Mechanism level: Failed to find any Kerberos tgt)])</h3>
<div class="paragraph">
<p>There can be several causes that produce this symptom.</p>
</div>
<div class="paragraph">
<p>First, check that you have a valid Kerberos ticket.
One is required in order to set up communication with a secure Apache HBase cluster.
Examine the ticket currently in the credential cache, if any, by running the <code>klist</code> command line utility.
If no ticket is listed, you must obtain a ticket by running the <code>kinit</code> command with either a keytab specified, or by interactively entering a password for the desired principal.</p>
</div>
<div class="paragraph">
<p>Then, consult the <a href="http://docs.oracle.com/javase/1.5.0/docs/guide/security/jgss/tutorials/Troubleshooting.html">Java Security Guide troubleshooting section</a>.
The most common problem addressed there is resolved by setting <code>javax.security.auth.useSubjectCredsOnly</code> system property value to <code>false</code>.</p>
</div>
<div class="paragraph">
<p>Because of a change in the format in which MIT Kerberos writes its credentials cache, there is a bug in the Oracle JDK 6 Update 26 and earlier that causes Java to be unable to read the Kerberos credentials cache created by versions of MIT Kerberos 1.8.1 or higher.
If you have this problematic combination of components in your environment, to work around this problem, first log in with <code>kinit</code> and then immediately refresh the credential cache with <code>kinit -R</code>.
The refresh will rewrite the credential cache without the problematic formatting.</p>
</div>
<div class="paragraph">
<p>Prior to JDK 1.4, the JCE was an unbundled product, and as such, the JCA and JCE were regularly referred to as separate, distinct components.
As JCE is now bundled in the JDK 7.0, the distinction is becoming less apparent. Since the JCE uses the same architecture as the JCA, the JCE should be more properly thought of as a part of the JCA.</p>
</div>
<div class="paragraph">
<p>You may need to install the <a href="https://docs.oracle.com/javase/1.5.0/docs/guide/security/jce/JCERefGuide.html">Java Cryptography Extension</a>, or JCE because of JDK 1.5 or earlier version.
Insure the JCE jars are on the classpath on both server and client systems.</p>
</div>
<div class="paragraph">
<p>You may also need to download the <a href="http://www.oracle.com/technetwork/java/javase/downloads/jce-6-download-429243.html">unlimited strength JCE policy files</a>.
Uncompress and extract the downloaded file, and install the policy jars into <em>&lt;java-home&gt;/lib/security</em>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.mapreduce"><a class="anchor" href="#trouble.mapreduce"></a>142. MapReduce</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.mapreduce.local"><a class="anchor" href="#trouble.mapreduce.local"></a>142.1. You Think You&#8217;re On The Cluster, But You&#8217;re Actually Local</h3>
<div class="paragraph">
<p>This following stacktrace happened using <code>ImportTsv</code>, but things like this can happen on any job with a mis-configuration.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="text"> WARN mapred.LocalJobRunner: job_local_0001
java.lang.IllegalArgumentException: Can't read partitions file
at org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner.setConf(TotalOrderPartitioner.java:111)
at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:62)
at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:117)
at org.apache.hadoop.mapred.MapTask$NewOutputCollector.&lt;init&gt;(MapTask.java:560)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:639)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:323)
at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:210)
Caused by: java.io.FileNotFoundException: File _partition.lst does not exist.
at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:383)
at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:251)
at org.apache.hadoop.fs.FileSystem.getLength(FileSystem.java:776)
at org.apache.hadoop.io.SequenceFile$Reader.&lt;init&gt;(SequenceFile.java:1424)
at org.apache.hadoop.io.SequenceFile$Reader.&lt;init&gt;(SequenceFile.java:1419)
at org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner.readPartitions(TotalOrderPartitioner.java:296)</code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;see the critical portion of the stack? It&#8217;s&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">at org.apache.hadoop.mapred.LocalJobRunner<span class="error">$</span>Job.run(LocalJobRunner.java:<span class="integer">210</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>LocalJobRunner means the job is running locally, not on the cluster.</p>
</div>
<div class="paragraph">
<p>To solve this problem, you should run your MR job with your <code>HADOOP_CLASSPATH</code> set to include the HBase dependencies.
The "hbase classpath" utility can be used to do this easily.
For example (substitute VERSION with your HBase version):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">HADOOP_CLASSPATH=`hbase classpath` hadoop jar $HBASE_HOME/hbase-mapreduce-VERSION.jar rowcounter usertable</code></pre>
</div>
</div>
<div class="paragraph">
<p>See <a href="#hbase.mapreduce.classpath">HBase, MapReduce, and the CLASSPATH</a> for more information on HBase MapReduce jobs and classpaths.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.hbasezerocopybytestring"><a class="anchor" href="#trouble.hbasezerocopybytestring"></a>142.2. Launching a job, you get java.lang.IllegalAccessError: com/google/protobuf/HBaseZeroCopyByteString or class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString</h3>
<div class="paragraph">
<p>See <a href="https://issues.apache.org/jira/browse/HBASE-10304">HBASE-10304 Running an hbase job jar: IllegalAccessError: class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString</a> and <a href="https://issues.apache.org/jira/browse/HBASE-11118">HBASE-11118 non environment variable solution for "IllegalAccessError: class com.google.protobuf.ZeroCopyLiteralByteString cannot access its superclass com.google.protobuf.LiteralByteString"</a>.
The issue can also show up when trying to run spark jobs.
See <a href="https://issues.apache.org/jira/browse/HBASE-10877">HBASE-10877 HBase non-retriable exception list should be expanded</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.namenode"><a class="anchor" href="#trouble.namenode"></a>143. NameNode</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For more information on the NameNode, see <a href="#arch.hdfs">HDFS</a>.</p>
</div>
<div class="sect2">
<h3 id="trouble.namenode.disk"><a class="anchor" href="#trouble.namenode.disk"></a>143.1. HDFS Utilization of Tables and Regions</h3>
<div class="paragraph">
<p>To determine how much space HBase is using on HDFS use the <code>hadoop</code> shell commands from the NameNode.
For example&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">hadoop fs -dus /hbase/</code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;returns the summarized disk utilization for all HBase objects.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">hadoop fs -dus /hbase/myTable</code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;returns the summarized disk utilization for the HBase table 'myTable'.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">hadoop fs -du /hbase/myTable</code></pre>
</div>
</div>
<div class="paragraph">
<p>&#8230;&#8203;returns a list of the regions under the HBase table 'myTable' and their disk utilization.</p>
</div>
<div class="paragraph">
<p>For more information on HDFS shell commands, see the <a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/FileSystemShell.html">HDFS FileSystem Shell documentation</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.namenode.hbase.objects"><a class="anchor" href="#trouble.namenode.hbase.objects"></a>143.2. Browsing HDFS for HBase Objects</h3>
<div class="paragraph">
<p>Sometimes it will be necessary to explore the HBase objects that exist on HDFS.
These objects could include the WALs (Write Ahead Logs), tables, regions, StoreFiles, etc.
The easiest way to do this is with the NameNode web application that runs on port 50070.
The NameNode web application will provide links to the all the DataNodes in the cluster so that they can be browsed seamlessly.</p>
</div>
<div class="paragraph">
<p>The HDFS directory structure of HBase tables in the cluster is&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">/hbase
/data
/&lt;Namespace&gt; (Namespaces in the cluster)
/&lt;Table&gt; (Tables in the cluster)
/&lt;<span class="predefined-type">Region</span>&gt; (Regions <span class="keyword">for</span> the table)
/&lt;ColumnFamily&gt; (ColumnFamilies <span class="keyword">for</span> the <span class="predefined-type">Region</span> <span class="keyword">for</span> the table)
/&lt;StoreFile&gt; (StoreFiles <span class="keyword">for</span> the ColumnFamily <span class="keyword">for</span> the Regions <span class="keyword">for</span> the table)</code></pre>
</div>
</div>
<div class="paragraph">
<p>The HDFS directory structure of HBase WAL is..</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">/hbase
/WALs
/&lt;RegionServer&gt; (RegionServers)
/&lt;WAL&gt; (WAL files <span class="keyword">for</span> the RegionServer)</code></pre>
</div>
</div>
<div class="paragraph">
<p>See the <a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html">HDFS User Guide</a> for other non-shell diagnostic utilities like <code>fsck</code>.</p>
</div>
<div class="sect3">
<h4 id="trouble.namenode.0size.hlogs"><a class="anchor" href="#trouble.namenode.0size.hlogs"></a>143.2.1. Zero size WALs with data in them</h4>
<div class="paragraph">
<p>Problem: when getting a listing of all the files in a RegionServer&#8217;s <em>WALs</em> directory, one file has a size of 0 but it contains data.</p>
</div>
<div class="paragraph">
<p>Answer: It&#8217;s an HDFS quirk.
A file that&#8217;s currently being written to will appear to have a size of 0 but once it&#8217;s closed it will show its true size</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.namenode.uncompaction"><a class="anchor" href="#trouble.namenode.uncompaction"></a>143.2.2. Use Cases</h4>
<div class="paragraph">
<p>Two common use-cases for querying HDFS for HBase objects is research the degree of uncompaction of a table.
If there are a large number of StoreFiles for each ColumnFamily it could indicate the need for a major compaction.
Additionally, after a major compaction if the resulting StoreFile is "small" it could indicate the need for a reduction of ColumnFamilies for the table.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_unexpected_filesystem_growth"><a class="anchor" href="#_unexpected_filesystem_growth"></a>143.3. Unexpected Filesystem Growth</h3>
<div class="paragraph">
<p>If you see an unexpected spike in filesystem usage by HBase, two possible culprits
are snapshots and WALs.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Snapshots</dt>
<dd>
<p>When you create a snapshot, HBase retains everything it needs to recreate the table&#8217;s
state at that time of the snapshot. This includes deleted cells or expired versions.
For this reason, your snapshot usage pattern should be well-planned, and you should
prune snapshots that you no longer need. Snapshots are stored in <code>/hbase/.hbase-snapshot</code>,
and archives needed to restore snapshots are stored in
<code>/hbase/archive/&lt;_tablename_&gt;/&lt;_region_&gt;/&lt;_column_family_&gt;/</code>.</p>
<div class="literalblock">
<div class="content">
<pre>*Do not* manage snapshots or archives manually via HDFS. HBase provides APIs and
HBase Shell commands for managing them. For more information, see &lt;&lt;ops.snapshots&gt;&gt;.</pre>
</div>
</div>
</dd>
<dt class="hdlist1">WAL</dt>
<dd>
<p>Write-ahead logs (WALs) are stored in subdirectories of the HBase root directory,
typically <code>/hbase/</code>, depending on their status. Already-processed WALs are stored
in <code>/hbase/oldWALs/</code> and corrupt WALs are stored in <code>/hbase/.corrupt/</code> for examination.
If the size of one of these subdirectories is growing, examine the HBase
server logs to find the root cause for why WALs are not being processed correctly.</p>
<div class="paragraph">
<p>If you use replication and <code>/hbase/oldWALs/</code> is using more space than you expect,
remember that WALs are saved when replication is disabled, as long as there are peers.</p>
</div>
</dd>
</dl>
</div>
<div class="paragraph">
<p><strong>Do not</strong> manage WALs manually via HDFS.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.network"><a class="anchor" href="#trouble.network"></a>144. Network</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.network.spikes"><a class="anchor" href="#trouble.network.spikes"></a>144.1. Network Spikes</h3>
<div class="paragraph">
<p>If you are seeing periodic network spikes you might want to check the <code>compactionQueues</code> to see if major compactions are happening.</p>
</div>
<div class="paragraph">
<p>See <a href="#managed.compactions">Managed Compactions</a> for more information on managing compactions.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.network.loopback"><a class="anchor" href="#trouble.network.loopback"></a>144.2. Loopback IP</h3>
<div class="paragraph">
<p>HBase expects the loopback IP Address to be 127.0.0.1.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.network.ints"><a class="anchor" href="#trouble.network.ints"></a>144.3. Network Interfaces</h3>
<div class="paragraph">
<p>Are all the network interfaces functioning correctly? Are you sure? See the Troubleshooting Case Study in <a href="#trouble.casestudy">Case Studies</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.rs"><a class="anchor" href="#trouble.rs"></a>145. RegionServer</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For more information on the RegionServers, see <a href="#regionserver.arch">RegionServer</a>.</p>
</div>
<div class="sect2">
<h3 id="trouble.rs.startup"><a class="anchor" href="#trouble.rs.startup"></a>145.1. Startup Errors</h3>
<div class="sect3">
<h4 id="trouble.rs.startup.master_no_region"><a class="anchor" href="#trouble.rs.startup.master_no_region"></a>145.1.1. Master Starts, But RegionServers Do Not</h4>
<div class="paragraph">
<p>The Master believes the RegionServers have the IP of 127.0.0.1 - which is localhost and resolves to the master&#8217;s own localhost.</p>
</div>
<div class="paragraph">
<p>The RegionServers are erroneously informing the Master that their IP addresses are 127.0.0.1.</p>
</div>
<div class="paragraph">
<p>Modify <em>/etc/hosts</em> on the region servers, from&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">#</span> Do not remove the following line, or various programs
<span class="error">#</span> that require network functionality will fail.
<span class="float">127.0</span><span class="float">.0</span><span class="float">.1</span> fully.qualified.regionservername regionservername localhost.localdomain localhost
::<span class="integer">1</span> localhost6.localdomain6 localhost6</code></pre>
</div>
</div>
<div class="paragraph">
<p>... to (removing the master node&#8217;s name from localhost)&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">#</span> Do not remove the following line, or various programs
<span class="error">#</span> that require network functionality will fail.
<span class="float">127.0</span><span class="float">.0</span><span class="float">.1</span> localhost.localdomain localhost
::<span class="integer">1</span> localhost6.localdomain6 localhost6</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.startup.compression"><a class="anchor" href="#trouble.rs.startup.compression"></a>145.1.2. Compression Link Errors</h4>
<div class="paragraph">
<p>Since compression algorithms such as LZO need to be installed and configured on each cluster this is a frequent source of startup error.
If you see messages like this&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">11</span>/<span class="octal">02</span>/<span class="integer">20</span> <span class="octal">01</span>:<span class="integer">32</span>:<span class="integer">15</span> ERROR lzo.GPLNativeCodeLoader: Could not load <span class="directive">native</span> gpl library
java.lang.UnsatisfiedLinkError: no gplcompression in java.library.path
at java.lang.ClassLoader.loadLibrary(<span class="predefined-type">ClassLoader</span>.java:<span class="integer">1734</span>)
at java.lang.Runtime.loadLibrary0(<span class="predefined-type">Runtime</span>.java:<span class="integer">823</span>)
at java.lang.System.loadLibrary(<span class="predefined-type">System</span>.java:<span class="integer">1028</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>... then there is a path issue with the compression libraries.
See the Configuration section on link:[LZO compression configuration].</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.startup.hsync"><a class="anchor" href="#trouble.rs.startup.hsync"></a>145.1.3. RegionServer aborts due to lack of hsync for filesystem</h4>
<div class="paragraph">
<p>In order to provide data durability for writes to the cluster HBase relies on the ability to durably save state in a write ahead log. When using a version of Apache Hadoop Common&#8217;s filesystem API that supports checking on the availability of needed calls, HBase will proactively abort the cluster if it finds it can&#8217;t operate safely.</p>
</div>
<div class="paragraph">
<p>For RegionServer roles, the failure will show up in logs like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-04-05 11:36:22,785 ERROR [regionserver/192.168.1.123:16020] wal.AsyncFSWALProvider: The RegionServer async write ahead log provider relies on the ability to call hflush and hsync for proper operation during component failures, but the current FileSystem does not support doing so. Please check the config value of 'hbase.wal.dir' and ensure it points to a FileSystem mount that has suitable capabilities for output streams.
2018-04-05 11:36:22,799 ERROR [regionserver/192.168.1.123:16020] regionserver.HRegionServer: ***** ABORTING region server 192.168.1.123,16020,1522946074234: Unhandled: cannot get log writer *****
java.io.IOException: cannot get log writer
at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:112)
at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:612)
at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:124)
at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:759)
at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:489)
at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.&lt;init&gt;(AsyncFSWAL.java:251)
at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createWAL(AsyncFSWALProvider.java:69)
at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createWAL(AsyncFSWALProvider.java:44)
at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWAL(AbstractFSWALProvider.java:138)
at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWAL(AbstractFSWALProvider.java:57)
at org.apache.hadoop.hbase.wal.WALFactory.getWAL(WALFactory.java:252)
at org.apache.hadoop.hbase.regionserver.HRegionServer.getWAL(HRegionServer.java:2105)
at org.apache.hadoop.hbase.regionserver.HRegionServer.buildServerLoad(HRegionServer.java:1326)
at org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerReport(HRegionServer.java:1191)
at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1007)
at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.hadoop.hbase.util.CommonFSUtils$StreamLacksCapabilityException: hflush and hsync
at org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper.createOutput(AsyncFSOutputHelper.java:69)
at org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.initOutput(AsyncProtobufLogWriter.java:168)
at org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:167)
at org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:99)
... 15 more</pre>
</div>
</div>
<div class="paragraph">
<p>If you are attempting to run in standalone mode and see this error, please walk back through the section <a href="#quickstart">Quick Start - Standalone HBase</a> and ensure you have included <strong>all</strong> the given configuration settings.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.startup.asyncfs"><a class="anchor" href="#trouble.rs.startup.asyncfs"></a>145.1.4. RegionServer aborts due to can not initialize access to HDFS</h4>
<div class="paragraph">
<p>We will try to use <em>AsyncFSWAL</em> for HBase-2.x as it has better performance while consuming less resources. But the problem for <em>AsyncFSWAL</em> is that it hacks into the internal of the DFSClient implementation, so it will easily be broken when upgrading hadoop, even for a simple patch release.</p>
</div>
<div class="paragraph">
<p>If you do not specify the wal provider, we will try to fall back to the old <em>FSHLog</em> if we fail to initialize <em>AsyncFSWAL</em>, but it may not always work. The failure will show up in logs like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>18/07/02 18:51:06 WARN concurrent.DefaultPromise: An exception was
thrown by org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper$13.operationComplete()
java.lang.Error: Couldn't properly initialize access to HDFS
internals. Please update your WAL Provider to not make use of the
'asyncfs' provider. See HBASE-16110 for more information.
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.&lt;clinit&gt;(FanOutOneBlockAsyncDFSOutputSaslHelper.java:268)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.initialize(FanOutOneBlockAsyncDFSOutputHelper.java:661)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.access$300(FanOutOneBlockAsyncDFSOutputHelper.java:118)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper$13.operationComplete(FanOutOneBlockAsyncDFSOutputHelper.java:720)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper$13.operationComplete(FanOutOneBlockAsyncDFSOutputHelper.java:715)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:507)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:500)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:479)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:420)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:104)
at org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:82)
at org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:638)
at org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:676)
at org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:552)
at org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:394)
at org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:304)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NoSuchMethodException:
org.apache.hadoop.hdfs.DFSClient.decryptEncryptedDataEncryptionKey(org.apache.hadoop.fs.FileEncryptionInfo)
at java.lang.Class.getDeclaredMethod(Class.java:2130)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.createTransparentCryptoHelper(FanOutOneBlockAsyncDFSOutputSaslHelper.java:232)
at org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper.&lt;clinit&gt;(FanOutOneBlockAsyncDFSOutputSaslHelper.java:262)
... 18 more</pre>
</div>
</div>
<div class="paragraph">
<p>If you hit this error, please specify <em>FSHLog</em>, i.e, <em>filesystem</em>, explicitly in your config file.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.wal.provider<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>filesystem<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>And do not forget to send an email to the <a href="mailto:user@hbase.apache.org">user@hbase.apache.org</a> or <a href="mailto:dev@hbase.apache.org">dev@hbase.apache.org</a> to report the failure and also your hadoop version, we will try to fix the problem ASAP in the next release.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.rs.runtime"><a class="anchor" href="#trouble.rs.runtime"></a>145.2. Runtime Errors</h3>
<div class="sect3">
<h4 id="trouble.rs.runtime.hang"><a class="anchor" href="#trouble.rs.runtime.hang"></a>145.2.1. RegionServer Hanging</h4>
<div class="paragraph">
<p>Are you running an old JVM (&lt; 1.6.0_u21?)? When you look at a thread dump, does it look like threads are BLOCKED but no one holds the lock all are blocked on? See <a href="https://issues.apache.org/jira/browse/HBASE-3622">HBASE 3622 Deadlock in
HBaseServer (JVM bug?)</a>.
Adding <code>-XX:+UseMembar</code> to the HBase <code>HBASE_OPTS</code> in <em>conf/hbase-env.sh</em> may fix it.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.filehandles"><a class="anchor" href="#trouble.rs.runtime.filehandles"></a>145.2.2. java.io.IOException&#8230;&#8203;(Too many open files)</h4>
<div class="paragraph">
<p>If you see log messages like this&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2010</span>-<span class="integer">09</span>-<span class="integer">13</span> <span class="octal">01</span>:<span class="integer">24</span>:<span class="integer">17</span>,<span class="integer">336</span> WARN org.apache.hadoop.hdfs.server.datanode.DataNode:
Disk-related <span class="exception">IOException</span> in BlockReceiver constructor. Cause is java.io.IOException: Too many open files
at java.io.UnixFileSystem.createFileExclusively(Native <span class="predefined-type">Method</span>)
at java.io.File.createNewFile(<span class="predefined-type">File</span>.java:<span class="integer">883</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>... see the Getting Started section on link:[ulimit and nproc configuration].</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.xceivers"><a class="anchor" href="#trouble.rs.runtime.xceivers"></a>145.2.3. xceiverCount 258 exceeds the limit of concurrent xcievers 256</h4>
<div class="paragraph">
<p>This typically shows up in the DataNode logs.</p>
</div>
<div class="paragraph">
<p>See the Getting Started section on link:[xceivers configuration].</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.oom_nt"><a class="anchor" href="#trouble.rs.runtime.oom_nt"></a>145.2.4. System instability, and the presence of "java.lang.OutOfMemoryError: unable to createnew native thread in exceptions" HDFS DataNode logs or that of any system daemon</h4>
<div class="paragraph">
<p>See the Getting Started section on ulimit and nproc configuration.
The default on recent Linux distributions is 1024 - which is far too low for HBase.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.gc"><a class="anchor" href="#trouble.rs.runtime.gc"></a>145.2.5. DFS instability and/or RegionServer lease timeouts</h4>
<div class="paragraph">
<p>If you see warning messages like this&#8230;&#8203;</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2009</span>-<span class="octal">02</span>-<span class="integer">24</span> <span class="integer">10</span>:<span class="octal">01</span>:<span class="integer">33</span>,<span class="integer">516</span> WARN org.apache.hadoop.hbase.util.Sleeper: We slept xxx ms, ten times longer than scheduled: <span class="integer">10000</span>
<span class="integer">2009</span>-<span class="octal">02</span>-<span class="integer">24</span> <span class="integer">10</span>:<span class="octal">01</span>:<span class="integer">33</span>,<span class="integer">516</span> WARN org.apache.hadoop.hbase.util.Sleeper: We slept xxx ms, ten times longer than scheduled: <span class="integer">15000</span>
<span class="integer">2009</span>-<span class="octal">02</span>-<span class="integer">24</span> <span class="integer">10</span>:<span class="octal">01</span>:<span class="integer">36</span>,<span class="integer">472</span> WARN org.apache.hadoop.hbase.regionserver.HRegionServer: unable to report to master <span class="keyword">for</span> xxx milliseconds - retrying</code></pre>
</div>
</div>
<div class="paragraph">
<p>... or see full GC compactions then you may be experiencing full GC&#8217;s.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.nolivenodes"><a class="anchor" href="#trouble.rs.runtime.nolivenodes"></a>145.2.6. "No live nodes contain current block" and/or YouAreDeadException</h4>
<div class="paragraph">
<p>These errors can happen either when running out of OS file handles or in periods of severe network problems where the nodes are unreachable.</p>
</div>
<div class="paragraph">
<p>See the Getting Started section on ulimit and nproc configuration and check your network.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.zkexpired"><a class="anchor" href="#trouble.rs.runtime.zkexpired"></a>145.2.7. ZooKeeper SessionExpired events</h4>
<div class="paragraph">
<p>Master or RegionServers shutting down with messages like those in the logs:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">WARN org.apache.zookeeper.ClientCnxn: <span class="exception">Exception</span>
closing session <span class="hex">0x278bd16a96000f</span> to sun.nio.ch.SelectionKeyImpl<span class="error">@</span><span class="integer">355811</span>ec
java.io.IOException: TIMED OUT
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.run(ClientCnxn.java:<span class="integer">906</span>)
WARN org.apache.hadoop.hbase.util.Sleeper: We slept <span class="integer">79410</span>ms, ten times longer than scheduled: <span class="integer">5000</span>
INFO org.apache.zookeeper.ClientCnxn: Attempting connection to server hostname/IP:PORT
INFO org.apache.zookeeper.ClientCnxn: Priming connection to java.nio.channels.SocketChannel[connected local=/IP:PORT remote=hostname/IP:PORT]
INFO org.apache.zookeeper.ClientCnxn: Server connection successful
WARN org.apache.zookeeper.ClientCnxn: <span class="exception">Exception</span> closing session <span class="hex">0x278bd16a96000d</span> to sun.nio.ch.SelectionKeyImpl<span class="error">@</span><span class="float">3544d</span><span class="integer">65</span>e
java.io.IOException: Session Expired
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.readConnectResult(ClientCnxn.java:<span class="integer">589</span>)
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.doIO(ClientCnxn.java:<span class="integer">709</span>)
at org.apache.zookeeper.ClientCnxn<span class="error">$</span>SendThread.run(ClientCnxn.java:<span class="integer">945</span>)
ERROR org.apache.hadoop.hbase.regionserver.HRegionServer: ZooKeeper session expired</code></pre>
</div>
</div>
<div class="paragraph">
<p>The JVM is doing a long running garbage collecting which is pausing every threads (aka "stop the world"). Since the RegionServer&#8217;s local ZooKeeper client cannot send heartbeats, the session times out.
By design, we shut down any node that isn&#8217;t able to contact the ZooKeeper ensemble after getting a timeout so that it stops serving data that may already be assigned elsewhere.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Make sure you give plenty of RAM (in <em>hbase-env.sh</em>), the default of 1GB won&#8217;t be able to sustain long running imports.</p>
</li>
<li>
<p>Make sure you don&#8217;t swap, the JVM never behaves well under swapping.</p>
</li>
<li>
<p>Make sure you are not CPU starving the RegionServer thread.
For example, if you are running a MapReduce job using 6 CPU-intensive tasks on a machine with 4 cores, you are probably starving the RegionServer enough to create longer garbage collection pauses.</p>
</li>
<li>
<p>Increase the ZooKeeper session timeout</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>If you wish to increase the session timeout, add the following to your <em>hbase-site.xml</em> to increase the timeout from the default of 60 seconds to 120 seconds.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>zookeeper.session.timeout<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>120000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.tickTime<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>6000<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Be aware that setting a higher timeout means that the regions served by a failed RegionServer will take at least that amount of time to be transferred to another RegionServer.
For a production system serving live requests, we would instead recommend setting it lower than 1 minute and over-provision your cluster in order the lower the memory load on each machines (hence having less garbage to collect per machine).</p>
</div>
<div class="paragraph">
<p>If this is happening during an upload which only happens once (like initially loading all your data into HBase), consider bulk loading.</p>
</div>
<div class="paragraph">
<p>See <a href="#trouble.zookeeper.general">ZooKeeper, The Cluster Canary</a> for other general information about ZooKeeper troubleshooting.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.notservingregion"><a class="anchor" href="#trouble.rs.runtime.notservingregion"></a>145.2.8. NotServingRegionException</h4>
<div class="paragraph">
<p>This exception is "normal" when found in the RegionServer logs at DEBUG level.
This exception is returned back to the client and then the client goes back to <code>hbase:meta</code> to find the new location of the moved region.</p>
</div>
<div class="paragraph">
<p>However, if the NotServingRegionException is logged ERROR, then the client ran out of retries and something probably wrong.</p>
</div>
</div>
<div class="sect3">
<h4 id="brand.new.compressor"><a class="anchor" href="#brand.new.compressor"></a>145.2.9. Logs flooded with '2011-01-10 12:40:48,407 INFO org.apache.hadoop.io.compress.CodecPool: Gotbrand-new compressor' messages</h4>
<div class="paragraph">
<p>We are not using the native versions of compression libraries.
See <a href="https://issues.apache.org/jira/browse/HBASE-1900">HBASE-1900 Put back native support when hadoop 0.21 is released</a>.
Copy the native libs from hadoop under HBase lib dir or symlink them into place and the message should go away.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.rs.runtime.client_went_away"><a class="anchor" href="#trouble.rs.runtime.client_went_away"></a>145.2.10. Server handler X on 60020 caught: java.nio.channels.ClosedChannelException</h4>
<div class="paragraph">
<p>If you see this type of message it means that the region server was trying to read/send data from/to a client but it already went away.
Typical causes for this are if the client was killed (you see a storm of messages like this when a MapReduce job is killed or fails) or if the client receives a SocketTimeoutException.
It&#8217;s harmless, but you should consider digging in a bit more if you aren&#8217;t doing something to trigger them.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_snapshot_errors_due_to_reverse_dns"><a class="anchor" href="#_snapshot_errors_due_to_reverse_dns"></a>145.3. Snapshot Errors Due to Reverse DNS</h3>
<div class="paragraph">
<p>Several operations within HBase, including snapshots, rely on properly configured reverse DNS.
Some environments, such as Amazon EC2, have trouble with reverse DNS.
If you see errors like the following on your RegionServers, check your reverse DNS configuration:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2013-05-01 00:04:56,356 DEBUG org.apache.hadoop.hbase.procedure.Subprocedure: Subprocedure 'backup1'
coordinator notified of 'acquire', waiting on 'reached' or 'abort' from coordinator.</pre>
</div>
</div>
<div class="paragraph">
<p>In general, the hostname reported by the RegionServer needs to be the same as the hostname the Master is trying to reach.
You can see a hostname mismatch by looking for the following type of message in the RegionServer&#8217;s logs at start-up.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2013-05-01 00:03:00,614 INFO org.apache.hadoop.hbase.regionserver.HRegionServer: Master passed us hostname
to use. Was=myhost-1234, Now=ip-10-55-88-99.ec2.internal</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.rs.shutdown"><a class="anchor" href="#trouble.rs.shutdown"></a>145.4. Shutdown Errors</h3>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.master"><a class="anchor" href="#trouble.master"></a>146. Master</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For more information on the Master, see <a href="#architecture.master">master</a>.</p>
</div>
<div class="sect2">
<h3 id="trouble.master.startup"><a class="anchor" href="#trouble.master.startup"></a>146.1. Startup Errors</h3>
<div class="sect3">
<h4 id="trouble.master.startup.migration"><a class="anchor" href="#trouble.master.startup.migration"></a>146.1.1. Master says that you need to run the HBase migrations script</h4>
<div class="paragraph">
<p>Upon running that, the HBase migrations script says no files in root directory.</p>
</div>
<div class="paragraph">
<p>HBase expects the root directory to either not exist, or to have already been initialized by HBase running a previous time.
If you create a new directory for HBase using Hadoop DFS, this error will occur.
Make sure the HBase root directory does not currently exist or has been initialized by a previous run of HBase.
Sure fire solution is to just use Hadoop dfs to delete the HBase root and let HBase create and initialize the directory itself.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.master.startup.zk.buffer"><a class="anchor" href="#trouble.master.startup.zk.buffer"></a>146.1.2. Packet len6080218 is out of range!</h4>
<div class="paragraph">
<p>If you have many regions on your cluster and you see an error like that reported above in this sections title in your logs, see <a href="https://issues.apache.org/jira/browse/HBASE-4246">HBASE-4246 Cluster with too many regions cannot withstand some master failover scenarios</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="trouble.master.startup.hsync"><a class="anchor" href="#trouble.master.startup.hsync"></a>146.1.3. Master fails to become active due to lack of hsync for filesystem</h4>
<div class="paragraph">
<p>HBase&#8217;s internal framework for cluster operations requires the ability to durably save state in a write ahead log. When using a version of Apache Hadoop Common&#8217;s filesystem API that supports checking on the availability of needed calls, HBase will proactively abort the cluster if it finds it can&#8217;t operate safely.</p>
</div>
<div class="paragraph">
<p>For Master roles, the failure will show up in logs like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-04-05 11:18:44,653 ERROR [Thread-21] master.HMaster: Failed to become active master
java.lang.IllegalStateException: The procedure WAL relies on the ability to hsync for proper operation during component failures, but the underlying filesystem does not support doing so. Please check the config value of 'hbase.procedure.store.wal.use.hsync' to set the desired level of robustness and ensure the config value of 'hbase.wal.dir' points to a FileSystem mount that can provide it.
at org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore.rollWriter(WALProcedureStore.java:1034)
at org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore.recoverLease(WALProcedureStore.java:374)
at org.apache.hadoop.hbase.procedure2.ProcedureExecutor.start(ProcedureExecutor.java:530)
at org.apache.hadoop.hbase.master.HMaster.startProcedureExecutor(HMaster.java:1267)
at org.apache.hadoop.hbase.master.HMaster.startServiceThreads(HMaster.java:1173)
at org.apache.hadoop.hbase.master.HMaster.finishActiveMasterInitialization(HMaster.java:881)
at org.apache.hadoop.hbase.master.HMaster.startActiveMasterManager(HMaster.java:2048)
at org.apache.hadoop.hbase.master.HMaster.lambda$run$0(HMaster.java:568)
at java.lang.Thread.run(Thread.java:745)</pre>
</div>
</div>
<div class="paragraph">
<p>If you are attempting to run in standalone mode and see this error, please walk back through the section <a href="#quickstart">Quick Start - Standalone HBase</a> and ensure you have included <strong>all</strong> the given configuration settings.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.master.shutdown"><a class="anchor" href="#trouble.master.shutdown"></a>146.2. Shutdown Errors</h3>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.zookeeper"><a class="anchor" href="#trouble.zookeeper"></a>147. ZooKeeper</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.zookeeper.startup"><a class="anchor" href="#trouble.zookeeper.startup"></a>147.1. Startup Errors</h3>
<div class="sect3">
<h4 id="trouble.zookeeper.startup.address"><a class="anchor" href="#trouble.zookeeper.startup.address"></a>147.1.1. Could not find my address: xyz in list of ZooKeeper quorum servers</h4>
<div class="paragraph">
<p>A ZooKeeper server wasn&#8217;t able to start, throws that error.
xyz is the name of your server.</p>
</div>
<div class="paragraph">
<p>This is a name lookup problem.
HBase tries to start a ZooKeeper server on some machine but that machine isn&#8217;t able to find itself in the <code>hbase.zookeeper.quorum</code> configuration.</p>
</div>
<div class="paragraph">
<p>Use the hostname presented in the error message instead of the value you used.
If you have a DNS server, you can set <code>hbase.zookeeper.dns.interface</code> and <code>hbase.zookeeper.dns.nameserver</code> in <em>hbase-site.xml</em> to make sure it resolves to the correct FQDN.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="trouble.zookeeper.general"><a class="anchor" href="#trouble.zookeeper.general"></a>147.2. ZooKeeper, The Cluster Canary</h3>
<div class="paragraph">
<p>ZooKeeper is the cluster&#8217;s "canary in the mineshaft". It&#8217;ll be the first to notice issues if any so making sure its happy is the short-cut to a humming cluster.</p>
</div>
<div class="paragraph">
<p>See the <a href="https://wiki.apache.org/hadoop/ZooKeeper/Troubleshooting">ZooKeeper Operating Environment Troubleshooting</a> page.
It has suggestions and tools for checking disk and networking performance; i.e.
the operating environment your ZooKeeper and HBase are running in.</p>
</div>
<div class="paragraph">
<p>Additionally, the utility <a href="#trouble.tools.builtin.zkcli">zkcli</a> may help investigate ZooKeeper issues.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.ec2"><a class="anchor" href="#trouble.ec2"></a>148. Amazon EC2</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.ec2.zookeeper"><a class="anchor" href="#trouble.ec2.zookeeper"></a>148.1. ZooKeeper does not seem to work on Amazon EC2</h3>
<div class="paragraph">
<p>HBase does not start when deployed as Amazon EC2 instances.
Exceptions like the below appear in the Master and/or RegionServer logs:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> <span class="integer">2009</span>-<span class="integer">10</span>-<span class="integer">19</span> <span class="integer">11</span>:<span class="integer">52</span>:<span class="integer">27</span>,<span class="octal">030</span> INFO org.apache.zookeeper.ClientCnxn: Attempting
connection to server ec2-<span class="integer">174</span>-<span class="integer">129</span>-<span class="integer">15</span>-<span class="integer">236</span>.compute-<span class="integer">1</span>.amazonaws.com/<span class="float">10.244</span><span class="float">.9</span><span class="float">.171</span>:<span class="integer">2181</span>
<span class="integer">2009</span>-<span class="integer">10</span>-<span class="integer">19</span> <span class="integer">11</span>:<span class="integer">52</span>:<span class="integer">27</span>,<span class="octal">032</span> WARN org.apache.zookeeper.ClientCnxn: <span class="exception">Exception</span>
closing session <span class="hex">0x0</span> to sun.nio.ch.SelectionKeyImpl<span class="error">@</span><span class="float">656d</span>c861
java.net.ConnectException: <span class="predefined-type">Connection</span> refused</code></pre>
</div>
</div>
<div class="paragraph">
<p>Security group policy is blocking the ZooKeeper port on a public address.
Use the internal EC2 host names when configuring the ZooKeeper quorum peer list.</p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.ec2.instability"><a class="anchor" href="#trouble.ec2.instability"></a>148.2. Instability on Amazon EC2</h3>
<div class="paragraph">
<p>Questions on HBase and Amazon EC2 come up frequently on the HBase dist-list.
Search for old threads using <a href="http://search-hadoop.com/">Search Hadoop</a></p>
</div>
</div>
<div class="sect2">
<h3 id="trouble.ec2.connection"><a class="anchor" href="#trouble.ec2.connection"></a>148.3. Remote Java Connection into EC2 Cluster Not Working</h3>
<div class="paragraph">
<p>See Andrew&#8217;s answer here, up on the user list: <a href="http://search-hadoop.com/m/sPdqNFAwyg2">Remote Java client connection into EC2 instance</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.versions"><a class="anchor" href="#trouble.versions"></a>149. HBase and Hadoop version issues</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.wrong.version"><a class="anchor" href="#trouble.wrong.version"></a>149.1. &#8230;&#8203;cannot communicate with client version&#8230;&#8203;</h3>
<div class="paragraph">
<p>If you see something like the following in your logs <span class="computeroutput">... 2012-09-24
10:20:52,168 FATAL org.apache.hadoop.hbase.master.HMaster: Unhandled exception. Starting
shutdown. org.apache.hadoop.ipc.RemoteException: Server IPC version 7 cannot communicate
with client version 4 ...</span> &#8230;&#8203;are you trying to talk to an Hadoop 2.0.x from an HBase that has an Hadoop 1.0.x client? Use the HBase built against Hadoop 2.0 or rebuild your HBase passing the -Dhadoop.profile=2.0 attribute to Maven (See <a href="#maven.build.hadoop">Building against various hadoop versions.</a> for more).</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_hbase_and_hdfs"><a class="anchor" href="#_hbase_and_hdfs"></a>150. HBase and HDFS</h2>
<div class="sectionbody">
<div class="paragraph">
<p>General configuration guidance for Apache HDFS is out of the scope of this guide.
Refer to the documentation available at <a href="https://hadoop.apache.org/" class="bare">https://hadoop.apache.org/</a> for extensive information about configuring HDFS.
This section deals with HDFS in terms of HBase.</p>
</div>
<div class="paragraph">
<p>In most cases, HBase stores its data in Apache HDFS.
This includes the HFiles containing the data, as well as the write-ahead logs (WALs) which store data before it is written to the HFiles and protect against RegionServer crashes.
HDFS provides reliability and protection to data in HBase because it is distributed.
To operate with the most efficiency, HBase needs data to be available locally.
Therefore, it is a good practice to run an HDFS DataNode on each RegionServer.</p>
</div>
<div class="dlist">
<div class="title">Important Information and Guidelines for HBase and HDFS</div>
<dl>
<dt class="hdlist1">HBase is a client of HDFS.</dt>
<dd>
<p>HBase is an HDFS client, using the HDFS <code>DFSClient</code> class, and references to this class appear in HBase logs with other HDFS client log messages.</p>
</dd>
<dt class="hdlist1">Configuration is necessary in multiple places.</dt>
<dd>
<p>Some HDFS configurations relating to HBase need to be done at the HDFS (server) side.
Others must be done within HBase (at the client side). Other settings need to be set at both the server and client side.</p>
</dd>
<dt class="hdlist1">Write errors which affect HBase may be logged in the HDFS logs rather than HBase logs.</dt>
<dd>
<p>When writing, HDFS pipelines communications from one DataNode to another.
HBase communicates to both the HDFS NameNode and DataNode, using the HDFS client classes.
Communication problems between DataNodes are logged in the HDFS logs, not the HBase logs.</p>
</dd>
<dt class="hdlist1">HBase communicates with HDFS using two different ports.</dt>
<dd>
<p>HBase communicates with DataNodes using the <code>ipc.Client</code> interface and the <code>DataNode</code> class.
References to these will appear in HBase logs.
Each of these communication channels use a different port (50010 and 50020 by default). The ports are configured in the HDFS configuration, via the <code>dfs.datanode.address</code> and <code>dfs.datanode.ipc.address</code> parameters.</p>
</dd>
<dt class="hdlist1">Errors may be logged in HBase, HDFS, or both.</dt>
<dd>
<p>When troubleshooting HDFS issues in HBase, check logs in both places for errors.</p>
</dd>
<dt class="hdlist1">HDFS takes a while to mark a node as dead. You can configure HDFS to avoid using stale DataNodes.</dt>
<dd>
<p>By default, HDFS does not mark a node as dead until it is unreachable for 630 seconds.
In Hadoop 1.1 and Hadoop 2.x, this can be alleviated by enabling checks for stale DataNodes, though this check is disabled by default.
You can enable the check for reads and writes separately, via <code>dfs.namenode.avoid.read.stale.datanode</code> and <code>dfs.namenode.avoid.write.stale.datanode settings</code>.
A stale DataNode is one that has not been reachable for <code>dfs.namenode.stale.datanode.interval</code> (default is 30 seconds). Stale datanodes are avoided, and marked as the last possible target for a read or write operation.
For configuration details, see the HDFS documentation.</p>
</dd>
<dt class="hdlist1">Settings for HDFS retries and timeouts are important to HBase.</dt>
<dd>
<p>You can configure settings for various retries and timeouts.
Always refer to the HDFS documentation for current recommendations and defaults.
Some of the settings important to HBase are listed here.
Defaults are current as of Hadoop 2.3.
Check the Hadoop documentation for the most current values and recommendations.</p>
</dd>
<dt class="hdlist1">The HBase Balancer and HDFS Balancer are incompatible</dt>
<dd>
<p>The HDFS balancer attempts to spread HDFS blocks evenly among DataNodes. HBase relies
on compactions to restore locality after a region split or failure. These two types
of balancing do not work well together.</p>
<div class="paragraph">
<p>In the past, the generally accepted advice was to turn off the HDFS load balancer and rely
on the HBase balancer, since the HDFS balancer would degrade locality. This advice
is still valid if your HDFS version is lower than 2.7.1.</p>
</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HDFS-6133">HDFS-6133</a> provides the ability
to exclude favored-nodes (pinned) blocks from the HDFS load balancer, by setting the
<code>dfs.datanode.block-pinning.enabled</code> property to <code>true</code> in the HDFS service
configuration.</p>
</div>
<div class="paragraph">
<p>HBase can be enabled to use the HDFS favored-nodes feature by switching the HBase balancer
class (conf: <code>hbase.master.loadbalancer.class</code>) to <code>org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer</code>
which is documented <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.html">here</a>.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
HDFS-6133 is available in HDFS 2.7.0 and higher, but HBase does not support
running on HDFS 2.7.0, so you must be using HDFS 2.7.1 or higher to use this feature
with HBase.
</td>
</tr>
</table>
</div>
</dd>
</dl>
</div>
<div class="paragraph">
<div class="title">Connection Timeouts</div>
<p>Connection timeouts occur between the client (HBASE) and the HDFS DataNode.
They may occur when establishing a connection, attempting to read, or attempting to write.
The two settings below are used in combination, and affect connections between the DFSClient and the DataNode, the ipc.cClient and the DataNode, and communication between two DataNodes.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>dfs.client.socket-timeout</code> (default: 60000)</dt>
<dd>
<p>The amount of time before a client connection times out when establishing a connection or reading.
The value is expressed in milliseconds, so the default is 60 seconds.</p>
</dd>
<dt class="hdlist1"><code>dfs.datanode.socket.write.timeout</code> (default: 480000)</dt>
<dd>
<p>The amount of time before a write operation times out.
The default is 8 minutes, expressed as milliseconds.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<div class="title">Typical Error Logs</div>
<p>The following types of errors are often seen in the logs.</p>
</div>
<div class="paragraph">
<p><code>INFO HDFS.DFSClient: Failed to connect to /xxx50010, add to deadNodes and
continue java.net.SocketTimeoutException: 60000 millis timeout while waiting for channel
to be ready for connect. ch : java.nio.channels.SocketChannel[connection-pending
remote=/region-server-1:50010]</code>::
All DataNodes for a block are dead, and recovery is not possible.
Here is the sequence of events that leads to this error:</p>
</div>
<div class="paragraph">
<p><code>INFO org.apache.hadoop.HDFS.DFSClient: Exception in createBlockOutputStream
java.net.SocketTimeoutException: 69000 millis timeout while waiting for channel to be
ready for connect. ch : java.nio.channels.SocketChannel[connection-pending remote=/
xxx:50010]</code>::
This type of error indicates a write issue.
In this case, the master wants to split the log.
It does not have a local DataNodes so it tries to connect to a remote DataNode, but the DataNode is dead.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.tests"><a class="anchor" href="#trouble.tests"></a>151. Running unit or integration tests</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.hdfs_2556"><a class="anchor" href="#trouble.hdfs_2556"></a>151.1. Runtime exceptions from MiniDFSCluster when running tests</h3>
<div class="paragraph">
<p>If you see something like the following</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
java.lang.NullPointerException: <span class="predefined-constant">null</span>
at org.apache.hadoop.hdfs.MiniDFSCluster.startDataNodes
at org.apache.hadoop.hdfs.MiniDFSCluster.&lt;init&gt;
at org.apache.hadoop.hbase.MiniHBaseCluster.&lt;init&gt;
at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniDFSCluster
at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster
...</code></pre>
</div>
</div>
<div class="paragraph">
<p>or</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
java.io.IOException: Shutting down
at org.apache.hadoop.hbase.MiniHBaseCluster.init
at org.apache.hadoop.hbase.MiniHBaseCluster.&lt;init&gt;
at org.apache.hadoop.hbase.MiniHBaseCluster.&lt;init&gt;
at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniHBaseCluster
at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster
...</code></pre>
</div>
</div>
<div class="paragraph">
<p>... then try issuing the command umask 022 before launching tests.
This is a workaround for <a href="https://issues.apache.org/jira/browse/HDFS-2556">HDFS-2556</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.casestudy"><a class="anchor" href="#trouble.casestudy"></a>152. Case Studies</h2>
<div class="sectionbody">
<div class="paragraph">
<p>For Performance and Troubleshooting Case Studies, see <a href="#casestudies">Apache HBase Case Studies</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="trouble.crypto"><a class="anchor" href="#trouble.crypto"></a>153. Cryptographic Features</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="trouble.crypto.hbase_10132"><a class="anchor" href="#trouble.crypto.hbase_10132"></a>153.1. sun.security.pkcs11.wrapper.PKCS11Exception: CKR_ARGUMENTS_BAD</h3>
<div class="paragraph">
<p>This problem manifests as exceptions ultimately caused by:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Caused by: sun.security.pkcs11.wrapper.PKCS11Exception: CKR_ARGUMENTS_BAD
at sun.security.pkcs11.wrapper.PKCS11.C_DecryptUpdate(Native <span class="predefined-type">Method</span>)
at sun.security.pkcs11.P11Cipher.implDoFinal(P11Cipher.java:<span class="integer">795</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>This problem appears to affect some versions of OpenJDK 7 shipped by some Linux vendors.
NSS is configured as the default provider.
If the host has an x86_64 architecture, depending on if the vendor packages contain the defect, the NSS provider will not function correctly.</p>
</div>
<div class="paragraph">
<p>To work around this problem, find the JRE home directory and edit the file <em>lib/security/java.security</em>.
Edit the file to comment out the line:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">security.provider<span class="float">.1</span>=sun.security.pkcs11.SunPKCS11 <span class="error">$</span>{java.home}/lib/security/nss.cfg</code></pre>
</div>
</div>
<div class="paragraph">
<p>Then renumber the remaining providers accordingly.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_operating_system_specific_issues"><a class="anchor" href="#_operating_system_specific_issues"></a>154. Operating System Specific Issues</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_page_allocation_failure"><a class="anchor" href="#_page_allocation_failure"></a>154.1. Page Allocation Failure</h3>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
This issue is known to affect CentOS 6.2 and possibly CentOS 6.5.
It may also affect some versions of Red Hat Enterprise Linux, according to <a href="https://bugzilla.redhat.com/show_bug.cgi?id=770545" class="bare">https://bugzilla.redhat.com/show_bug.cgi?id=770545</a>.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Some users have reported seeing the following error:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>kernel: java: page allocation failure. order:4, mode:0x20</pre>
</div>
</div>
<div class="paragraph">
<p>Raising the value of <code>min_free_kbytes</code> was reported to fix this problem.
This parameter is set to a percentage of the amount of RAM on your system, and is described in more detail at <a href="http://www.centos.org/docs/5/html/5.1/Deployment_Guide/s3-proc-sys-vm.html" class="bare">http://www.centos.org/docs/5/html/5.1/Deployment_Guide/s3-proc-sys-vm.html</a>.</p>
</div>
<div class="paragraph">
<p>To find the current value on your system, run the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>[user@host]# cat /proc/sys/vm/min_free_kbytes</pre>
</div>
</div>
<div class="paragraph">
<p>Next, raise the value.
Try doubling, then quadrupling the value.
Note that setting the value too low or too high could have detrimental effects on your system.
Consult your operating system vendor for specific recommendations.</p>
</div>
<div class="paragraph">
<p>Use the following command to modify the value of <code>min_free_kbytes</code>, substituting <em>&lt;value&gt;</em> with your intended value:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>[user@host]# echo &lt;value&gt; &gt; /proc/sys/vm/min_free_kbytes</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_jdk_issues"><a class="anchor" href="#_jdk_issues"></a>155. JDK Issues</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_nosuchmethoderror_java_util_concurrent_concurrenthashmap_keyset"><a class="anchor" href="#_nosuchmethoderror_java_util_concurrent_concurrenthashmap_keyset"></a>155.1. NoSuchMethodError: java.util.concurrent.ConcurrentHashMap.keySet</h3>
<div class="paragraph">
<p>If you see this in your logs:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Caused by: java.lang.NoSuchMethodError: java.util.concurrent.ConcurrentHashMap.keySet()Ljava/util/concurrent/<span class="predefined-type">ConcurrentHashMap</span><span class="error">$</span>KeySetView;
at org.apache.hadoop.hbase.master.ServerManager.findServerWithSameHostnamePortWithLock(ServerManager.java:<span class="integer">393</span>)
at org.apache.hadoop.hbase.master.ServerManager.checkAndRecordNewServer(ServerManager.java:<span class="integer">307</span>)
at org.apache.hadoop.hbase.master.ServerManager.regionServerStartup(ServerManager.java:<span class="integer">244</span>)
at org.apache.hadoop.hbase.master.MasterRpcServices.regionServerStartup(MasterRpcServices.java:<span class="integer">304</span>)
at org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos<span class="error">$</span>RegionServerStatusService<span class="error">$</span><span class="integer">2</span>.callBlockingMethod(RegionServerStatusProtos.java:<span class="integer">7910</span>)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:<span class="integer">2020</span>)
... <span class="integer">4</span> more</code></pre>
</div>
</div>
<div class="paragraph">
<p>then check if you compiled with jdk8 and tried to run it on jdk7.
If so, this won&#8217;t work.
Run on jdk8 or recompile with jdk7.
See <a href="https://issues.apache.org/jira/browse/HBASE-10607">HBASE-10607 JDK8 NoSuchMethodError involving ConcurrentHashMap.keySet if running on JRE 7</a>.</p>
</div>
</div>
</div>
</div>
<h1 id="casestudies" class="sect0"><a class="anchor" href="#casestudies"></a>Apache HBase Case Studies</h1>
<div class="sect1">
<h2 id="casestudies.overview"><a class="anchor" href="#casestudies.overview"></a>156. Overview</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This chapter will describe a variety of performance and troubleshooting case studies that can provide a useful blueprint on diagnosing Apache HBase cluster issues.</p>
</div>
<div class="paragraph">
<p>For more information on Performance and Troubleshooting, see <a href="#performance">Apache HBase Performance Tuning</a> and <a href="#trouble">Troubleshooting and Debugging Apache HBase</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="casestudies.schema"><a class="anchor" href="#casestudies.schema"></a>157. Schema Design</h2>
<div class="sectionbody">
<div class="paragraph">
<p>See the schema design case studies here: <a href="#schema.casestudies">Schema Design Case Studies</a></p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="casestudies.perftroub"><a class="anchor" href="#casestudies.perftroub"></a>158. Performance/Troubleshooting</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="casestudies.slownode"><a class="anchor" href="#casestudies.slownode"></a>158.1. Case Study #1 (Performance Issue On A Single Node)</h3>
<div class="sect3">
<h4 id="_scenario"><a class="anchor" href="#_scenario"></a>158.1.1. Scenario</h4>
<div class="paragraph">
<p>Following a scheduled reboot, one data node began exhibiting unusual behavior.
Routine MapReduce jobs run against HBase tables which regularly completed in five or six minutes began taking 30 or 40 minutes to finish.
These jobs were consistently found to be waiting on map and reduce tasks assigned to the troubled data node (e.g., the slow map tasks all had the same Input Split). The situation came to a head during a distributed copy, when the copy was severely prolonged by the lagging node.</p>
</div>
</div>
<div class="sect3">
<h4 id="_hardware"><a class="anchor" href="#_hardware"></a>158.1.2. Hardware</h4>
<div class="ulist">
<div class="title">Datanodes:</div>
<ul>
<li>
<p>Two 12-core processors</p>
</li>
<li>
<p>Six Enterprise SATA disks</p>
</li>
<li>
<p>24GB of RAM</p>
</li>
<li>
<p>Two bonded gigabit NICs</p>
</li>
</ul>
</div>
<div class="ulist">
<div class="title">Network:</div>
<ul>
<li>
<p>10 Gigabit top-of-rack switches</p>
</li>
<li>
<p>20 Gigabit bonded interconnects between racks.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_hypotheses"><a class="anchor" href="#_hypotheses"></a>158.1.3. Hypotheses</h4>
<div class="sect4">
<h5 id="_hbase_hot_spot_region"><a class="anchor" href="#_hbase_hot_spot_region"></a>HBase "Hot Spot" Region</h5>
<div class="paragraph">
<p>We hypothesized that we were experiencing a familiar point of pain: a "hot spot" region in an HBase table, where uneven key-space distribution can funnel a huge number of requests to a single HBase region, bombarding the RegionServer process and cause slow response time.
Examination of the HBase Master status page showed that the number of HBase requests to the troubled node was almost zero.
Further, examination of the HBase logs showed that there were no region splits, compactions, or other region transitions in progress.
This effectively ruled out a "hot spot" as the root cause of the observed slowness.</p>
</div>
</div>
<div class="sect4">
<h5 id="_hbase_region_with_non_local_data"><a class="anchor" href="#_hbase_region_with_non_local_data"></a>HBase Region With Non-Local Data</h5>
<div class="paragraph">
<p>Our next hypothesis was that one of the MapReduce tasks was requesting data from HBase that was not local to the DataNode, thus forcing HDFS to request data blocks from other servers over the network.
Examination of the DataNode logs showed that there were very few blocks being requested over the network, indicating that the HBase region was correctly assigned, and that the majority of the necessary data was located on the node.
This ruled out the possibility of non-local data causing a slowdown.</p>
</div>
</div>
<div class="sect4">
<h5 id="_excessive_i_o_wait_due_to_swapping_or_an_over_worked_or_failing_hard_disk"><a class="anchor" href="#_excessive_i_o_wait_due_to_swapping_or_an_over_worked_or_failing_hard_disk"></a>Excessive I/O Wait Due To Swapping Or An Over-Worked Or Failing Hard Disk</h5>
<div class="paragraph">
<p>After concluding that the Hadoop and HBase were not likely to be the culprits, we moved on to troubleshooting the DataNode&#8217;s hardware.
Java, by design, will periodically scan its entire memory space to do garbage collection.
If system memory is heavily overcommitted, the Linux kernel may enter a vicious cycle, using up all of its resources swapping Java heap back and forth from disk to RAM as Java tries to run garbage collection.
Further, a failing hard disk will often retry reads and/or writes many times before giving up and returning an error.
This can manifest as high iowait, as running processes wait for reads and writes to complete.
Finally, a disk nearing the upper edge of its performance envelope will begin to cause iowait as it informs the kernel that it cannot accept any more data, and the kernel queues incoming data into the dirty write pool in memory.
However, using <code>vmstat(1)</code> and <code>free(1)</code>, we could see that no swap was being used, and the amount of disk IO was only a few kilobytes per second.</p>
</div>
</div>
<div class="sect4">
<h5 id="_slowness_due_to_high_processor_usage"><a class="anchor" href="#_slowness_due_to_high_processor_usage"></a>Slowness Due To High Processor Usage</h5>
<div class="paragraph">
<p>Next, we checked to see whether the system was performing slowly simply due to very high computational load. <code>top(1)</code> showed that the system load was higher than normal, but <code>vmstat(1)</code> and <code>mpstat(1)</code> showed that the amount of processor being used for actual computation was low.</p>
</div>
</div>
<div class="sect4">
<h5 id="_network_saturation_the_winner"><a class="anchor" href="#_network_saturation_the_winner"></a>Network Saturation (The Winner)</h5>
<div class="paragraph">
<p>Since neither the disks nor the processors were being utilized heavily, we moved on to the performance of the network interfaces.
The DataNode had two gigabit ethernet adapters, bonded to form an active-standby interface. <code>ifconfig(8)</code> showed some unusual anomalies, namely interface errors, overruns, framing errors.
While not unheard of, these kinds of errors are exceedingly rare on modern hardware which is operating as it should:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ /sbin/ifconfig bond0
bond0 Link encap:Ethernet HWaddr 00:00:00:00:00:00
inet addr:10.x.x.x Bcast:10.x.x.255 Mask:255.255.255.0
UP BROADCAST RUNNING MASTER MULTICAST MTU:1500 Metric:1
RX packets:2990700159 errors:12 dropped:0 overruns:1 frame:6 &lt;--- Look Here! Errors!
TX packets:3443518196 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:0
RX bytes:2416328868676 (2.4 TB) TX bytes:3464991094001 (3.4 TB)</pre>
</div>
</div>
<div class="paragraph">
<p>These errors immediately lead us to suspect that one or more of the ethernet interfaces might have negotiated the wrong line speed.
This was confirmed both by running an ICMP ping from an external host and observing round-trip-time in excess of 700ms, and by running <code>ethtool(8)</code> on the members of the bond interface and discovering that the active interface was operating at 100Mbs/, full duplex.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ sudo ethtool eth0
Settings for eth0:
Supported ports: [ TP ]
Supported link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Supports auto-negotiation: Yes
Advertised link modes: 10baseT/Half 10baseT/Full
100baseT/Half 100baseT/Full
1000baseT/Full
Advertised pause frame use: No
Advertised auto-negotiation: Yes
Link partner advertised link modes: Not reported
Link partner advertised pause frame use: No
Link partner advertised auto-negotiation: No
Speed: 100Mb/s &lt;--- Look Here! Should say 1000Mb/s!
Duplex: Full
Port: Twisted Pair
PHYAD: 1
Transceiver: internal
Auto-negotiation: on
MDI-X: Unknown
Supports Wake-on: umbg
Wake-on: g
Current message level: 0x00000003 (3)
Link detected: yes</pre>
</div>
</div>
<div class="paragraph">
<p>In normal operation, the ICMP ping round trip time should be around 20ms, and the interface speed and duplex should read, "1000MB/s", and, "Full", respectively.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_resolution"><a class="anchor" href="#_resolution"></a>158.1.4. Resolution</h4>
<div class="paragraph">
<p>After determining that the active ethernet adapter was at the incorrect speed, we used the <code>ifenslave(8)</code> command to make the standby interface the active interface, which yielded an immediate improvement in MapReduce performance, and a 10 times improvement in network throughput:</p>
</div>
<div class="paragraph">
<p>On the next trip to the datacenter, we determined that the line speed issue was ultimately caused by a bad network cable, which was replaced.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="casestudies.perf.1"><a class="anchor" href="#casestudies.perf.1"></a>158.2. Case Study #2 (Performance Research 2012)</h3>
<div class="paragraph">
<p>Investigation results of a self-described "we&#8217;re not sure what&#8217;s wrong, but it seems slow" problem. <a href="http://gbif.blogspot.com/2012/03/hbase-performance-evaluation-continued.html" class="bare">http://gbif.blogspot.com/2012/03/hbase-performance-evaluation-continued.html</a></p>
</div>
</div>
<div class="sect2">
<h3 id="casestudies.perf.2"><a class="anchor" href="#casestudies.perf.2"></a>158.3. Case Study #3 (Performance Research 2010))</h3>
<div class="paragraph">
<p>Investigation results of general cluster performance from 2010.
Although this research is on an older version of the codebase, this writeup is still very useful in terms of approach. <a href="http://hstack.org/hbase-performance-testing/" class="bare">http://hstack.org/hbase-performance-testing/</a></p>
</div>
</div>
<div class="sect2">
<h3 id="casestudies.max.transfer.threads"><a class="anchor" href="#casestudies.max.transfer.threads"></a>158.4. Case Study #4 (max.transfer.threads Config)</h3>
<div class="paragraph">
<p>Case study of configuring <code>max.transfer.threads</code> (previously known as <code>xcievers</code>) and diagnosing errors from misconfigurations. <a href="http://www.larsgeorge.com/2012/03/hadoop-hbase-and-xceivers.html" class="bare">http://www.larsgeorge.com/2012/03/hadoop-hbase-and-xceivers.html</a></p>
</div>
<div class="paragraph">
<p>See also <a href="#dfs.datanode.max.transfer.threads"><code>dfs.datanode.max.transfer.threads</code> </a>.</p>
</div>
</div>
</div>
</div>
<h1 id="ops_mgt" class="sect0"><a class="anchor" href="#ops_mgt"></a>Apache HBase Operational Management</h1>
<div class="openblock partintro">
<div class="content">
This chapter will cover operational tools and practices required of a running Apache HBase cluster.
The subject of operations is related to the topics of <a href="#trouble">Troubleshooting and Debugging Apache HBase</a>, <a href="#performance">Apache HBase Performance Tuning</a>, and <a href="#configuration">Apache HBase Configuration</a> but is a distinct topic in itself.
</div>
</div>
<div class="sect1">
<h2 id="tools"><a class="anchor" href="#tools"></a>159. HBase Tools and Utilities</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase provides several tools for administration, analysis, and debugging of your cluster.
The entry-point to most of these tools is the <em>bin/hbase</em> command, though some tools are available in the <em>dev-support/</em> directory.</p>
</div>
<div class="paragraph">
<p>To see usage instructions for <em>bin/hbase</em> command, run it with no arguments, or with the <code>-h</code> argument.
These are the usage instructions for HBase 0.98.x.
Some commands, such as <code>version</code>, <code>pe</code>, <code>ltt</code>, <code>clean</code>, are not available in previous versions.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase
Usage: hbase [&lt;options&gt;] &lt;command&gt; [&lt;args&gt;]
Options:
--config DIR Configuration direction to use. Default: ./conf
--hosts HOSTS Override the list in 'regionservers' file
--auth-as-server Authenticate to ZooKeeper using servers configuration
Commands:
Some commands take arguments. Pass no args or -h for usage.
shell Run the HBase shell
hbck Run the HBase 'fsck' tool. Defaults read-only hbck1.
Pass '-j /path/to/HBCK2.jar' to run hbase-2.x HBCK2.
snapshot Tool for managing snapshots
wal Write-ahead-log analyzer
hfile Store file analyzer
zkcli Run the ZooKeeper shell
master Run an HBase HMaster node
regionserver Run an HBase HRegionServer node
zookeeper Run a ZooKeeper server
rest Run an HBase REST server
thrift Run the HBase Thrift server
thrift2 Run the HBase Thrift2 server
clean Run the HBase clean up script
classpath Dump hbase CLASSPATH
mapredcp Dump CLASSPATH entries required by mapreduce
pe Run PerformanceEvaluation
ltt Run LoadTestTool
canary Run the Canary tool
version Print the version
backup Backup tables for recovery
restore Restore tables from existing backup image
regionsplitter Run RegionSplitter tool
rowcounter Run RowCounter tool
cellcounter Run CellCounter tool
CLASSNAME Run the class named CLASSNAME</pre>
</div>
</div>
<div class="paragraph">
<p>Some of the tools and utilities below are Java classes which are passed directly to the <em>bin/hbase</em> command, as referred to in the last line of the usage instructions.
Others, such as <code>hbase shell</code> (<a href="#shell">The Apache HBase Shell</a>), <code>hbase upgrade</code> (<a href="#upgrading">Upgrading</a>), and <code>hbase thrift</code> (<a href="#thrift">Thrift API and Filter Language</a>), are documented elsewhere in this guide.</p>
</div>
<div class="sect2">
<h3 id="_canary"><a class="anchor" href="#_canary"></a>159.1. Canary</h3>
<div class="paragraph">
<p>The Canary tool can help users "canary-test" the HBase cluster status.
The default "region mode" fetches a row from every column-family of every regions.
In "regionserver mode", the Canary tool will fetch a row from a random
region on each of the cluster&#8217;s RegionServers. In "zookeeper mode", the
Canary will read the root znode on each member of the zookeeper ensemble.</p>
</div>
<div class="paragraph">
<p>To see usage, pass the <code>-help</code> parameter (if you pass no
parameters, the Canary tool starts executing in the default
region "mode" fetching a row from every region in the cluster).</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-10-16 13:11:27,037 INFO [main] tool.Canary: Execution thread count=16
Usage: canary [OPTIONS] [&lt;TABLE1&gt; [&lt;TABLE2]...] | [&lt;REGIONSERVER1&gt; [&lt;REGIONSERVER2]..]
Where [OPTIONS] are:
-h,-help show this help and exit.
-regionserver set 'regionserver mode'; gets row from random region on server
-allRegions get from ALL regions when 'regionserver mode', not just random one.
-zookeeper set 'zookeeper mode'; grab zookeeper.znode.parent on each ensemble member
-daemon continuous check at defined intervals.
-interval &lt;N&gt; interval between checks in seconds
-e consider table/regionserver argument as regular expression
-f &lt;B&gt; exit on first error; default=true
-failureAsError treat read/write failure as error
-t &lt;N&gt; timeout for canary-test run; default=600000ms
-writeSniffing enable write sniffing
-writeTable the table used for write sniffing; default=hbase:canary
-writeTableTimeout &lt;N&gt; timeout for writeTable; default=600000ms
-readTableTimeouts &lt;tableName&gt;=&lt;read timeout&gt;,&lt;tableName&gt;=&lt;read timeout&gt;,...
comma-separated list of table read timeouts (no spaces);
logs 'ERROR' if takes longer. default=600000ms
-permittedZookeeperFailures &lt;N&gt; Ignore first N failures attempting to
connect to individual zookeeper nodes in ensemble
-D&lt;configProperty&gt;=&lt;value&gt; to assign or override configuration params
-Dhbase.canary.read.raw.enabled=&lt;true/false&gt; Set to enable/disable raw scan; default=false
Canary runs in one of three modes: region (default), regionserver, or zookeeper.
To sniff/probe all regions, pass no arguments.
To sniff/probe all regions of a table, pass tablename.
To sniff/probe regionservers, pass -regionserver, etc.
See http://hbase.apache.org/book.html#_canary for Canary documentation.</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
The <code>Sink</code> class is instantiated using the <code>hbase.canary.sink.class</code> configuration property.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>This tool will return non zero error codes to user for collaborating with other monitoring tools,
such as Nagios. The error code definitions are:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">int</span> USAGE_EXIT_CODE = <span class="integer">1</span>;
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">int</span> INIT_ERROR_EXIT_CODE = <span class="integer">2</span>;
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">int</span> TIMEOUT_ERROR_EXIT_CODE = <span class="integer">3</span>;
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">int</span> ERROR_EXIT_CODE = <span class="integer">4</span>;
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">int</span> FAILURE_EXIT_CODE = <span class="integer">5</span>;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Here are some examples based on the following given case: given two Table objects called test-01
and test-02 each with two column family cf1 and cf2 respectively, deployed on 3 RegionServers.
See the following table.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">RegionServer</th>
<th class="tableblock halign-left valign-top">test-01</th>
<th class="tableblock halign-left valign-top">test-02</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">rs1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">r1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">r2</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">rs2</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">r2</p></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">rs3</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">r2</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">r1</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following are some example outputs based on the previous given case.</p>
</div>
<div class="sect3">
<h4 id="_canary_test_for_every_column_family_store_of_every_region_of_every_table"><a class="anchor" href="#_canary_test_for_every_column_family_store_of_every_region_of_every_table"></a>159.1.1. Canary test for every column family (store) of every region of every table</h4>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary
3/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf1 in 2ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,,1386230156732.0e3c7d77ffb6361ea1b996ac1042ca9a. column family cf2 in 2ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,0004883,1386230156732.87b55e03dfeade00f441125159f8ca87. column family cf1 in 4ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-01,0004883,1386230156732.87b55e03dfeade00f441125159f8ca87. column family cf2 in 1ms
...
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,,1386559511167.aa2951a86289281beee480f107bb36ee. column family cf1 in 5ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,,1386559511167.aa2951a86289281beee480f107bb36ee. column family cf2 in 3ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,0004883,1386559511167.cbda32d5e2e276520712d84eaaa29d84. column family cf1 in 31ms
13/12/09 03:26:32 INFO tool.Canary: read from region test-02,0004883,1386559511167.cbda32d5e2e276520712d84eaaa29d84. column family cf2 in 8ms</pre>
</div>
</div>
<div class="paragraph">
<p>So you can see, table test-01 has two regions and two column families, so the Canary tool in the
default "region mode" will pick 4 small piece of data from 4 (2 region * 2 store) different stores.
This is a default behavior.</p>
</div>
</div>
<div class="sect3">
<h4 id="_canary_test_for_every_column_family_store_of_every_region_of_a_specific_table_s"><a class="anchor" href="#_canary_test_for_every_column_family_store_of_every_region_of_a_specific_table_s"></a>159.1.2. Canary test for every column family (store) of every region of a specific table(s)</h4>
<div class="paragraph">
<p>You can also test one or more specific tables by passing table names.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary test-01 test-02</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_canary_test_with_regionserver_granularity"><a class="anchor" href="#_canary_test_with_regionserver_granularity"></a>159.1.3. Canary test with RegionServer granularity</h4>
<div class="paragraph">
<p>In "regionserver mode", the Canary tool will pick one small piece of data
from each RegionServer (You can also pass one or more RegionServer names as arguments
to the canary-test when in "regionserver mode").</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -regionserver
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-01 on region server:rs2 in 72ms
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-02 on region server:rs3 in 34ms
13/12/09 06:05:17 INFO tool.Canary: Read from table:test-01 on region server:rs1 in 56ms</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_canary_test_with_regular_expression_pattern"><a class="anchor" href="#_canary_test_with_regular_expression_pattern"></a>159.1.4. Canary test with regular expression pattern</h4>
<div class="paragraph">
<p>You can pass regexes for table names when in "region mode" or for servernames when
in "regionserver mode". The below will test both table test-01 and test-02.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -e test-0[1-2]</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_run_canary_test_as_a_daemon"><a class="anchor" href="#_run_canary_test_as_a_daemon"></a>159.1.5. Run canary test as a "daemon"</h4>
<div class="paragraph">
<p>Run repeatedly with an interval defined via the option <code>-interval</code> (default value is 60 seconds).
This daemon will stop itself and return non-zero error code if any error occur. To have
the daemon keep running across errors, pass the -f flag with its value set to false
(see usage above).</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -daemon</pre>
</div>
</div>
<div class="paragraph">
<p>To run repeatedly with 5 second intervals and not stop on errors, do the following.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -daemon -interval 5 -f false</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_force_timeout_if_canary_test_stuck"><a class="anchor" href="#_force_timeout_if_canary_test_stuck"></a>159.1.6. Force timeout if canary test stuck</h4>
<div class="paragraph">
<p>In some cases the request is stuck and no response is sent back to the client. This
can happen with dead RegionServers which the master has not yet noticed.
Because of this we provide a timeout option to kill the canary test and return a
non-zero error code. The below sets the timeout value to 60 seconds (the default value
is 600 seconds).</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -t 60000</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_enable_write_sniffing_in_canary"><a class="anchor" href="#_enable_write_sniffing_in_canary"></a>159.1.7. Enable write sniffing in canary</h4>
<div class="paragraph">
<p>By default, the canary tool only checks read operations. To enable the write sniffing,
you can run the canary with the <code>-writeSniffing</code> option set. When write sniffing is
enabled, the canary tool will create an hbase table and make sure the
regions of the table are distributed to all region servers. In each sniffing period,
the canary will try to put data to these regions to check the write availability of
each region server.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -writeSniffing</pre>
</div>
</div>
<div class="paragraph">
<p>The default write table is <code>hbase:canary</code> and can be specified with the option <code>-writeTable</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -writeSniffing -writeTable ns:canary</pre>
</div>
</div>
<div class="paragraph">
<p>The default value size of each put is 10 bytes. You can set it via the config key:
<code>hbase.canary.write.value.size</code>.</p>
</div>
</div>
<div class="sect3">
<h4 id="_treat_read_write_failure_as_error"><a class="anchor" href="#_treat_read_write_failure_as_error"></a>159.1.8. Treat read / write failure as error</h4>
<div class="paragraph">
<p>By default, the canary tool only logs read failures&#8201;&#8212;&#8201;due to e.g. RetriesExhaustedException, etc.&#8201;&#8212;&#8201;and will return the 'normal' exit code. To treat read/write failure as errors, you can run canary
with the <code>-treatFailureAsError</code> option. When enabled, read/write failures will result in an
error exit code.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ${HBASE_HOME}/bin/hbase canary -treatFailureAsError</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_running_canary_in_a_kerberos_enabled_cluster"><a class="anchor" href="#_running_canary_in_a_kerberos_enabled_cluster"></a>159.1.9. Running Canary in a Kerberos-enabled Cluster</h4>
<div class="paragraph">
<p>To run the Canary in a Kerberos-enabled cluster, configure the following two properties in <em>hbase-site.xml</em>:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.client.keytab.file</code></p>
</li>
<li>
<p><code>hbase.client.kerberos.principal</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Kerberos credentials are refreshed every 30 seconds when Canary runs in daemon mode.</p>
</div>
<div class="paragraph">
<p>To configure the DNS interface for the client, configure the following optional properties in <em>hbase-site.xml</em>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.client.dns.interface</code></p>
</li>
<li>
<p><code>hbase.client.dns.nameserver</code></p>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 43. Canary in a Kerberos-Enabled Cluster</div>
<div class="content">
<div class="paragraph">
<p>This example shows each of the properties with valid values.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.kerberos.principal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hbase/_HOST@YOUR-REALM.COM<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.keytab.file<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>/etc/hbase/conf/keytab.krb5<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="comment">&lt;!-- optional params --&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.dns.interface<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.client.dns.nameserver<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>default<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_regionsplitter"><a class="anchor" href="#_regionsplitter"></a>159.2. RegionSplitter</h3>
<div class="listingblock">
<div class="content">
<pre>usage: bin/hbase regionsplitter &lt;TABLE&gt; &lt;SPLITALGORITHM&gt;
SPLITALGORITHM is the java class name of a class implementing
SplitAlgorithm, or one of the special strings
HexStringSplit or DecimalStringSplit or
UniformSplit, which are built-in split algorithms.
HexStringSplit treats keys as hexadecimal ASCII, and
DecimalStringSplit treats keys as decimal ASCII, and
UniformSplit treats keys as arbitrary bytes.
-c &lt;region count&gt; Create a new table with a pre-split number of
regions
-D &lt;property=value&gt; Override HBase Configuration Settings
-f &lt;family:family:...&gt; Column Families to create with new table.
Required with -c
--firstrow &lt;arg&gt; First Row in Table for Split Algorithm
-h Print this usage help
--lastrow &lt;arg&gt; Last Row in Table for Split Algorithm
-o &lt;count&gt; Max outstanding splits that have unfinished
major compactions
-r Perform a rolling split of an existing region
--risky Skip verification steps to complete
quickly. STRONGLY DISCOURAGED for production
systems.</pre>
</div>
</div>
<div class="paragraph">
<p>For additional detail, see <a href="#manual_region_splitting_decisions">Manual Region Splitting</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="health.check"><a class="anchor" href="#health.check"></a>159.3. Health Checker</h3>
<div class="paragraph">
<p>You can configure HBase to run a script periodically and if it fails N times (configurable), have the server exit.
See <em>HBASE-7351 Periodic health check script</em> for configurations and detail.</p>
</div>
</div>
<div class="sect2">
<h3 id="_driver"><a class="anchor" href="#_driver"></a>159.4. Driver</h3>
<div class="paragraph">
<p>Several frequently-accessed utilities are provided as <code>Driver</code> classes, and executed by the <em>bin/hbase</em> command.
These utilities represent MapReduce jobs which run on your cluster.
They are run in the following way, replacing <em>UtilityName</em> with the utility you want to run.
This command assumes you have set the environment variable <code>HBASE_HOME</code> to the directory where HBase is unpacked on your server.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.mapreduce.UtilityName</pre>
</div>
</div>
<div class="paragraph">
<p>The following utilities are available:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>LoadIncrementalHFiles</code></dt>
<dd>
<p>Complete a bulk data load.</p>
</dd>
<dt class="hdlist1"><code>CopyTable</code></dt>
<dd>
<p>Export a table from the local cluster to a peer cluster.</p>
</dd>
<dt class="hdlist1"><code>Export</code></dt>
<dd>
<p>Write table data to HDFS.</p>
</dd>
<dt class="hdlist1"><code>Import</code></dt>
<dd>
<p>Import data written by a previous <code>Export</code> operation.</p>
</dd>
<dt class="hdlist1"><code>ImportTsv</code></dt>
<dd>
<p>Import data in TSV format.</p>
</dd>
<dt class="hdlist1"><code>RowCounter</code></dt>
<dd>
<p>Count rows in an HBase table.</p>
</dd>
<dt class="hdlist1"><code>CellCounter</code></dt>
<dd>
<p>Count cells in an HBase table.</p>
</dd>
<dt class="hdlist1"><code>replication.VerifyReplication</code></dt>
<dd>
<p>Compare the data from tables in two different clusters.
WARNING: It doesn&#8217;t work for incrementColumnValues&#8217;d cells since the timestamp is changed.
Note that this command is in a different package than the others.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>Each command except <code>RowCounter</code> and <code>CellCounter</code> accept a single <code>--help</code> argument to print usage instructions.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbck"><a class="anchor" href="#hbck"></a>159.5. HBase <code>hbck</code></h3>
<div class="paragraph">
<p>The <code>hbck</code> tool that shipped with hbase-1.x has been made read-only in hbase-2.x. It is not able to repair
hbase-2.x clusters as hbase internals have changed. Nor should its assessments in read-only mode be
trusted as it does not understand hbase-2.x operation.</p>
</div>
<div class="paragraph">
<p>A new tool, <a href="#HBCK2">HBase <code>HBCK2</code></a>, described in the next section, replaces <code>hbck</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="HBCK2"><a class="anchor" href="#HBCK2"></a>159.6. HBase <code>HBCK2</code></h3>
<div class="paragraph">
<p><code>HBCK2</code> is the successor to <a href="#hbck">HBase <code>hbck</code></a>, the hbase-1.x fix tool (A.K.A <code>hbck1</code>). Use it in place of <code>hbck1</code>
making repairs against hbase-2.x installs.</p>
</div>
<div class="paragraph">
<p><code>HBCK2</code> does not ship as part of hbase. It can be found as a subproject of the companion
<a href="https://github.com/apache/hbase-operator-tools">hbase-operator-tools</a> repository at
<a href="https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2">Apache HBase HBCK2 Tool</a>.
<code>HBCK2</code> was moved out of hbase so it could evolve at a cadence apart from that of hbase core.</p>
</div>
<div class="paragraph">
<p>See the [<a href="https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2" class="bare">https://github.com/apache/hbase-operator-tools/tree/master/hbase-hbck2</a>](HBCK2) Home Page
for how <code>HBCK2</code> differs from <code>hbck1</code>, and for how to build and use it.</p>
</div>
<div class="paragraph">
<p>Once built, you can run <code>HBCK2</code> as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> hbase hbck -j /path/to/HBCK2.jar</code></pre>
</div>
</div>
<div class="paragraph">
<p>This will generate <code>HBCK2</code> usage describing commands and options.</p>
</div>
</div>
<div class="sect2">
<h3 id="hfile_tool2"><a class="anchor" href="#hfile_tool2"></a>159.7. HFile Tool</h3>
<div class="paragraph">
<p>See <a href="#hfile_tool">HFile Tool</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_wal_tools"><a class="anchor" href="#_wal_tools"></a>159.8. WAL Tools</h3>
<div class="sect3">
<h4 id="hlog_tool"><a class="anchor" href="#hlog_tool"></a>159.8.1. FSHLog tool</h4>
<div class="paragraph">
<p>The main method on <code>FSHLog</code> offers manual split and dump facilities.
Pass it WALs or the product of a split, the content of the <em>recovered.edits</em>.
directory.</p>
</div>
<div class="paragraph">
<p>You can get a textual dump of a WAL file content by doing the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre> $ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.FSHLog --dump hdfs://example.org:8020/hbase/WALs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012</pre>
</div>
</div>
<div class="paragraph">
<p>The return code will be non-zero if there are any issues with the file so you can test wholesomeness of file by redirecting <code>STDOUT</code> to <code>/dev/null</code> and testing the program return.</p>
</div>
<div class="paragraph">
<p>Similarly you can force a split of a log file directory by doing:</p>
</div>
<div class="listingblock">
<div class="content">
<pre> $ ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.FSHLog --split hdfs://example.org:8020/hbase/WALs/example.org,60020,1283516293161/</pre>
</div>
</div>
<div class="sect4">
<h5 id="hlog_tool.prettyprint"><a class="anchor" href="#hlog_tool.prettyprint"></a>WALPrettyPrinter</h5>
<div class="paragraph">
<p>The <code>WALPrettyPrinter</code> is a tool with configurable options to print the contents of a WAL.
You can invoke it via the HBase cli with the 'wal' command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre> $ ./bin/hbase wal hdfs://example.org:8020/hbase/WALs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">WAL Printing in older versions of HBase</div>
<div class="paragraph">
<p>Prior to version 2.0, the <code>WALPrettyPrinter</code> was called the <code>HLogPrettyPrinter</code>, after an internal name for HBase&#8217;s write ahead log.
In those versions, you can print the contents of a WAL using the same configuration as above, but with the 'hlog' command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre> $ ./bin/hbase hlog hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012</pre>
</div>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="compression.tool"><a class="anchor" href="#compression.tool"></a>159.9. Compression Tool</h3>
<div class="paragraph">
<p>See <a href="#compression.test">compression.test</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="copy.table"><a class="anchor" href="#copy.table"></a>159.10. CopyTable</h3>
<div class="paragraph">
<p>CopyTable is a utility that can copy part or of all of a table, either to the same cluster or another cluster.
The target table must first exist.
The usage is as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --help
/bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --help
Usage: CopyTable [general options] [--starttime=X] [--endtime=Y] [--new.name=NEW] [--peer.adr=ADR] &lt;tablename&gt;
Options:
rs.class hbase.regionserver.class of the peer cluster,
specify if different from current cluster
rs.impl hbase.regionserver.impl of the peer cluster,
startrow the start row
stoprow the stop row
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
versions number of cell versions to copy
new.name new table's name
peer.adr Address of the peer cluster given in the format
hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent
families comma-separated list of families to copy
To copy from cf1 to cf2, give sourceCfName:destCfName.
To keep the same name, just give "cfName"
all.cells also copy delete markers and deleted cells
Args:
tablename Name of the table to copy
Examples:
To copy 'TestTable' to a cluster that uses replication for a 1 hour window:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.CopyTable --starttime=1265875194289 --endtime=1265878794289 --peer.adr=server1,server2,server3:2181:/hbase --families=myOldCf:myNewCf,cf2,cf3 TestTable
For performance consider the following general options:
It is recommended that you set the following to &gt;=100. A higher value uses more memory but
decreases the round trip time to the server and may increase performance.
-Dhbase.client.scanner.caching=100
The following should always be set to false, to prevent writing data twice, which may produce
inaccurate results.
-Dmapred.map.tasks.speculative.execution=false</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Scanner Caching</div>
<div class="paragraph">
<p>Caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the job configuration.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Versions</div>
<div class="paragraph">
<p>By default, CopyTable utility only copies the latest version of row cells unless <code>--versions=n</code> is explicitly specified in the command.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Data Load</div>
<div class="paragraph">
<p>CopyTable does not perform a diff, it copies all Cells in between the specified startrow/stoprow starttime/endtime range.
This means that already existing cells with same values will still be copied.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>See Jonathan Hsieh&#8217;s <a href="https://blog.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/">Online
HBase Backups with CopyTable</a> blog post for more on <code>CopyTable</code>.</p>
</div>
</div>
<div class="sect2">
<h3 id="hashtable.synctable"><a class="anchor" href="#hashtable.synctable"></a>159.11. HashTable/SyncTable</h3>
<div class="paragraph">
<p>HashTable/SyncTable is a two steps tool for synchronizing table data, where each of the steps are implemented as MapReduce jobs.
Similarly to CopyTable, it can be used for partial or entire table data syncing, under same or remote cluster.
However, it performs the sync in a more efficient way than CopyTable. Instead of copying all cells
in specified row key/time period range, HashTable (the first step) creates hashed indexes for batch of cells on source table and output those as results.
On the next stage, SyncTable scans the source table and now calculates hash indexes for table cells,
compares these hashes with the outputs of HashTable, then it just scans (and compares) cells for diverging hashes, only updating
mismatching cells. This results in less network traffic/data transfers, which can be impacting when syncing large tables on remote clusters.</p>
</div>
<div class="sect3">
<h4 id="_step_1_hashtable"><a class="anchor" href="#_step_1_hashtable"></a>159.11.1. Step 1, HashTable</h4>
<div class="paragraph">
<p>First, run HashTable on the source table cluster (this is the table whose state will be copied to its counterpart).</p>
</div>
<div class="paragraph">
<p>Usage:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase org.apache.hadoop.hbase.mapreduce.HashTable --help
Usage: HashTable [options] &lt;tablename&gt; &lt;outputpath&gt;
Options:
batchsize the target amount of bytes to hash in each batch
rows are added to the batch until this size is reached
(defaults to 8000 bytes)
numhashfiles the number of hash files to create
if set to fewer than number of regions then
the job will create this number of reducers
(defaults to 1/100 of regions -- at least 1)
startrow the start row
stoprow the stop row
starttime beginning of the time range (unixtime in millis)
without endtime means from starttime to forever
endtime end of the time range. Ignored if no starttime specified.
scanbatch scanner batch size to support intra row scans
versions number of cell versions to include
families comma-separated list of families to include
Args:
tablename Name of the table to hash
outputpath Filesystem path to put the output data
Examples:
To hash 'TestTable' in 32kB batches for a 1 hour window into 50 files:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.HashTable --batchsize=32000 --numhashfiles=50 --starttime=1265875194289 --endtime=1265878794289 --families=cf2,cf3 TestTable /hashes/testTable</pre>
</div>
</div>
<div class="paragraph">
<p>The <strong>batchsize</strong> property defines how much cell data for a given region will be hashed together in a single hash value.
Sizing this properly has a direct impact on the sync efficiency, as it may lead to less scans executed by mapper tasks
of SyncTable (the next step in the process). The rule of thumb is that, the smaller the number of cells out of sync
(lower probability of finding a diff), larger batch size values can be determined.</p>
</div>
</div>
<div class="sect3">
<h4 id="_step_2_synctable"><a class="anchor" href="#_step_2_synctable"></a>159.11.2. Step 2, SyncTable</h4>
<div class="paragraph">
<p>Once HashTable has completed on source cluster, SyncTable can be ran on target cluster.
Just like replication and other synchronization jobs, it requires that all RegionServers/DataNodes
on source cluster be accessible by NodeManagers on the target cluster (where SyncTable job tasks will be running).</p>
</div>
<div class="paragraph">
<p>Usage:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase org.apache.hadoop.hbase.mapreduce.SyncTable --help
Usage: SyncTable [options] &lt;sourcehashdir&gt; &lt;sourcetable&gt; &lt;targettable&gt;
Options:
sourcezkcluster ZK cluster key of the source table
(defaults to cluster in classpath's config)
targetzkcluster ZK cluster key of the target table
(defaults to cluster in classpath's config)
dryrun if true, output counters but no writes
(defaults to false)
doDeletes if false, does not perform deletes
(defaults to true)
doPuts if false, does not perform puts
(defaults to true)
Args:
sourcehashdir path to HashTable output dir for source table
(see org.apache.hadoop.hbase.mapreduce.HashTable)
sourcetable Name of the source table to sync from
targettable Name of the target table to sync to
Examples:
For a dry run SyncTable of tableA from a remote source cluster
to a local target cluster:
$ bin/hbase org.apache.hadoop.hbase.mapreduce.SyncTable --dryrun=true --sourcezkcluster=zk1.example.com,zk2.example.com,zk3.example.com:2181:/hbase hdfs://nn:9000/hashes/tableA tableA tableA</pre>
</div>
</div>
<div class="paragraph">
<p>The <strong>dryrun</strong> option is useful when a read only, diff report is wanted, as it will produce only COUNTERS indicating the differences, but will not perform
any actual changes. It can be used as an alternative to VerifyReplication tool.</p>
</div>
<div class="paragraph">
<p>By default, SyncTable will cause target table to become an exact copy of source table (at least, for the specified startrow/stoprow or/and starttime/endtime).</p>
</div>
<div class="paragraph">
<p>Setting doDeletes to false modifies default behaviour to not delete target cells that are missing on source.
Similarly, setting doPuts to false modifies default behaviour to not add missing cells on target. Setting both doDeletes
and doPuts to false would give same effect as setting dryrun to true.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Additional info on doDeletes/doPuts</div>
<div class="paragraph">
<p>"doDeletes/doPuts" were only added by
<a href="https://jira.apache.org/jira/browse/HBASE-20305">HBASE-20305</a>, so these may not be available on
all released versions.
For major 1.x versions, minimum minor release including it is <strong>1.4.10</strong>.
For major 2.x versions, minimum minor release including it is <strong>2.1.5</strong>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Set doDeletes to false on Two-Way Replication scenarios</div>
<div class="paragraph">
<p>On Two-Way Replication or other scenarios where both source and target clusters can have data ingested, it&#8217;s advisable to always set doDeletes option to false,
as any additional cell inserted on SyncTable target cluster and not yet replicated to source would be deleted, and potentially lost permanently.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Set sourcezkcluster to the actual source cluster ZK quorum</div>
<div class="paragraph">
<p>Although not required, if sourcezkcluster is not set, SyncTable will connect to local HBase cluster for both source and target,
which does not give any meaningful result.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Remote Clusters on different Kerberos Realms</div>
<div class="paragraph">
<p>Often, remote clusters may be deployed on different Kerberos Realms.
<a href="https://jira.apache.org/jira/browse/HBASE-20586">HBASE-20586</a> added SyncTable support for
cross realm authentication, allowing a SyncTable process running on target cluster to connect to
source cluster and read both HashTable output files and the given HBase table when performing the
required comparisons.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="export"><a class="anchor" href="#export"></a>159.12. Export</h3>
<div class="paragraph">
<p>Export is a utility that will dump the contents of table to HDFS in a sequence file.
The Export can be run via a Coprocessor Endpoint or MapReduce. Invoke via:</p>
</div>
<div class="paragraph">
<p><strong>mapreduce-based Export</strong></p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.mapreduce.Export &lt;tablename&gt; &lt;outputdir&gt; [&lt;versions&gt; [&lt;starttime&gt; [&lt;endtime&gt;]]]</pre>
</div>
</div>
<div class="paragraph">
<p><strong>endpoint-based Export</strong></p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Make sure the Export coprocessor is enabled by adding <code>org.apache.hadoop.hbase.coprocessor.Export</code> to <code>hbase.coprocessor.region.classes</code>.
</td>
</tr>
</table>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.coprocessor.Export &lt;tablename&gt; &lt;outputdir&gt; [&lt;versions&gt; [&lt;starttime&gt; [&lt;endtime&gt;]]]</pre>
</div>
</div>
<div class="paragraph">
<p>The outputdir is a HDFS directory that does not exist prior to the export. When done, the exported files will be owned by the user invoking the export command.</p>
</div>
<div class="paragraph">
<p><strong>The Comparison of Endpoint-based Export And Mapreduce-based Export</strong></p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top"></th>
<th class="tableblock halign-left valign-top">Endpoint-based Export</th>
<th class="tableblock halign-left valign-top">Mapreduce-based Export</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">HBase version requirement</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">2.0+</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">0.2.1+</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Maven dependency</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase-endpoint</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hbase-mapreduce (2.0+), hbase-server(prior to 2.0)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Requirement before dump</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">mount the endpoint.Export on the target table</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">deploy the MapReduce framework</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Read latency</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">low, directly read the data from region</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">normal, traditional RPC scan</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Read Scalability</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">depend on number of regions</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">depend on number of mappers (see TableInputFormatBase#getSplits)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Timeout</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">operation timeout. configured by hbase.client.operation.timeout</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">scan timeout. configured by hbase.client.scanner.timeout.period</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Permission requirement</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">READ, EXECUTE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">READ</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Fault tolerance</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">no</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">depend on MapReduce</p></td>
</tr>
</tbody>
</table>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
To see usage instructions, run the command with no options. Available options include
specifying column families and applying filters during the export.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>By default, the <code>Export</code> tool only exports the newest version of a given cell, regardless of the number of versions stored. To export more than one version, replace <strong><em>&lt;versions&gt;</em></strong> with the desired number of versions.</p>
</div>
<div class="paragraph">
<p>Note: caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the job configuration.</p>
</div>
</div>
<div class="sect2">
<h3 id="import"><a class="anchor" href="#import"></a>159.13. Import</h3>
<div class="paragraph">
<p>Import is a utility that will load data that has been exported back into HBase.
Invoke via:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.mapreduce.Import &lt;tablename&gt; &lt;inputdir&gt;</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
To see usage instructions, run the command with no options.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>To import 0.94 exported files in a 0.96 cluster or onwards, you need to set system property "hbase.import.version" when running the import command as below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase -Dhbase.import.version=0.94 org.apache.hadoop.hbase.mapreduce.Import &lt;tablename&gt; &lt;inputdir&gt;</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="importtsv"><a class="anchor" href="#importtsv"></a>159.14. ImportTsv</h3>
<div class="paragraph">
<p>ImportTsv is a utility that will load data in TSV format into HBase.
It has two distinct usages: loading data from TSV format in HDFS into HBase via Puts, and preparing StoreFiles to be loaded via the <code>completebulkload</code>.</p>
</div>
<div class="paragraph">
<p>To load data via Puts (i.e., non-bulk loading):</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.mapreduce.ImportTsv -Dimporttsv.columns=a,b,c &lt;tablename&gt; &lt;hdfs-inputdir&gt;</pre>
</div>
</div>
<div class="paragraph">
<p>To generate StoreFiles for bulk-loading:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ bin/hbase org.apache.hadoop.hbase.mapreduce.ImportTsv -Dimporttsv.columns=a,b,c -Dimporttsv.bulk.output=hdfs://storefile-outputdir &lt;tablename&gt; &lt;hdfs-data-inputdir&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>These generated StoreFiles can be loaded into HBase via <a href="#completebulkload">completebulkload</a>.</p>
</div>
<div class="sect3">
<h4 id="importtsv.options"><a class="anchor" href="#importtsv.options"></a>159.14.1. ImportTsv Options</h4>
<div class="paragraph">
<p>Running <code>ImportTsv</code> with no arguments prints brief usage information:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Usage: importtsv -Dimporttsv.columns=a,b,c &lt;tablename&gt; &lt;inputdir&gt;
Imports the given input directory of TSV data into the specified table.
The column names of the TSV data must be specified using the -Dimporttsv.columns
option. This option takes the form of comma-separated column names, where each
column name is either a simple column family, or a columnfamily:qualifier. The special
column name HBASE_ROW_KEY is used to designate that this column should be used
as the row key for each imported record. You must specify exactly one column
to be the row key, and you must specify a column name for every column that exists in the
input data.
By default importtsv will load data directly into HBase. To instead generate
HFiles of data to prepare for a bulk data load, pass the option:
-Dimporttsv.bulk.output=/path/for/output
Note: the target table will be created with default column family descriptors if it does not already exist.
Other options that may be specified with -D include:
-Dimporttsv.skip.bad.lines=false - fail if encountering an invalid line
'-Dimporttsv.separator=|' - eg separate on pipes instead of tabs
-Dimporttsv.timestamp=currentTimeAsLong - use the specified timestamp for the import
-Dimporttsv.mapper.class=my.Mapper - A user-defined Mapper to use instead of org.apache.hadoop.hbase.mapreduce.TsvImporterMapper</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="importtsv.example"><a class="anchor" href="#importtsv.example"></a>159.14.2. ImportTsv Example</h4>
<div class="paragraph">
<p>For example, assume that we are loading data into a table called 'datatsv' with a ColumnFamily called 'd' with two columns "c1" and "c2".</p>
</div>
<div class="paragraph">
<p>Assume that an input file exists as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>row1 c1 c2
row2 c1 c2
row3 c1 c2
row4 c1 c2
row5 c1 c2
row6 c1 c2
row7 c1 c2
row8 c1 c2
row9 c1 c2
row10 c1 c2</pre>
</div>
</div>
<div class="paragraph">
<p>For ImportTsv to use this input file, the command line needs to look like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre> HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-mapreduce-VERSION.jar importtsv -Dimporttsv.columns=HBASE_ROW_KEY,d:c1,d:c2 -Dimporttsv.bulk.output=hdfs://storefileoutput datatsv hdfs://inputfile</pre>
</div>
</div>
<div class="paragraph">
<p>... and in this example the first column is the rowkey, which is why the HBASE_ROW_KEY is used.
The second and third columns in the file will be imported as "d:c1" and "d:c2", respectively.</p>
</div>
</div>
<div class="sect3">
<h4 id="importtsv.warning"><a class="anchor" href="#importtsv.warning"></a>159.14.3. ImportTsv Warning</h4>
<div class="paragraph">
<p>If you have preparing a lot of data for bulk loading, make sure the target HBase table is pre-split appropriately.</p>
</div>
</div>
<div class="sect3">
<h4 id="importtsv.also"><a class="anchor" href="#importtsv.also"></a>159.14.4. See Also</h4>
<div class="paragraph">
<p>For more information about bulk-loading HFiles into HBase, see <a href="#arch.bulk.load">arch.bulk.load</a></p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="completebulkload"><a class="anchor" href="#completebulkload"></a>159.15. CompleteBulkLoad</h3>
<div class="paragraph">
<p>The <code>completebulkload</code> utility will move generated StoreFiles into an HBase table.
This utility is often used in conjunction with output from <a href="#importtsv">importtsv</a>.</p>
</div>
<div class="paragraph">
<p>There are two ways to invoke this utility, with explicit classname and via the driver:</p>
</div>
<div class="listingblock">
<div class="title">Explicit Classname</div>
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles &lt;hdfs://storefileoutput&gt; &lt;tablename&gt;</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Driver</div>
<div class="content">
<pre>HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/hbase-mapreduce-VERSION.jar completebulkload &lt;hdfs://storefileoutput&gt; &lt;tablename&gt;</pre>
</div>
</div>
<div class="sect3">
<h4 id="completebulkload.warning"><a class="anchor" href="#completebulkload.warning"></a>159.15.1. CompleteBulkLoad Warning</h4>
<div class="paragraph">
<p>Data generated via MapReduce is often created with file permissions that are not compatible with the running HBase process.
Assuming you&#8217;re running HDFS with permissions enabled, those permissions will need to be updated before you run CompleteBulkLoad.</p>
</div>
<div class="paragraph">
<p>For more information about bulk-loading HFiles into HBase, see <a href="#arch.bulk.load">arch.bulk.load</a>.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="walplayer"><a class="anchor" href="#walplayer"></a>159.16. WALPlayer</h3>
<div class="paragraph">
<p>WALPlayer is a utility to replay WAL files into HBase.</p>
</div>
<div class="paragraph">
<p>The WAL can be replayed for a set of tables or all tables, and a timerange can be provided (in milliseconds). The WAL is filtered to this set of tables.
The output can optionally be mapped to another set of tables.</p>
</div>
<div class="paragraph">
<p>WALPlayer can also generate HFiles for later bulk importing, in that case only a single table and no mapping can be specified.</p>
</div>
<div class="paragraph">
<p>Invoke via:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer [options] &lt;wal inputdir&gt; &lt;tables&gt; [&lt;tableMappings&gt;]&gt;</pre>
</div>
</div>
<div class="paragraph">
<p>For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer /backuplogdir oldTable1,oldTable2 newTable1,newTable2</pre>
</div>
</div>
<div class="paragraph">
<p>WALPlayer, by default, runs as a mapreduce job.
To NOT run WALPlayer as a mapreduce job on your cluster, force it to run all in the local process by adding the flags <code>-Dmapreduce.jobtracker.address=local</code> on the command line.</p>
</div>
<div class="sect3">
<h4 id="walplayer.options"><a class="anchor" href="#walplayer.options"></a>159.16.1. WALPlayer Options</h4>
<div class="paragraph">
<p>Running <code>WALPlayer</code> with no arguments prints brief usage information:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Usage: WALPlayer [options] &lt;wal inputdir&gt; &lt;tables&gt; [&lt;tableMappings&gt;]
Replay all WAL files into HBase.
&lt;tables&gt; is a comma separated list of tables.
If no tables ("") are specified, all tables are imported.
(Be careful, hbase:meta entries will be imported in this case.)
WAL entries can be mapped to new set of tables via &lt;tableMappings&gt;.
&lt;tableMappings&gt; is a comma separated list of target tables.
If specified, each table in &lt;tables&gt; must have a mapping.
By default WALPlayer will load data directly into HBase.
To generate HFiles for a bulk data load instead, pass the following option:
-Dwal.bulk.output=/path/for/output
(Only one table can be specified, and no mapping is allowed!)
Time range options:
-Dwal.start.time=[date|ms]
-Dwal.end.time=[date|ms]
(The start and the end date of timerange. The dates can be expressed
in milliseconds since epoch or in yyyy-MM-dd'T'HH:mm:ss.SS format.
E.g. 1234567890120 or 2009-02-13T23:32:30.12)
Other options:
-Dmapreduce.job.name=jobName
Use the specified mapreduce job name for the wal player
For performance also consider the following options:
-Dmapreduce.map.speculative=false
-Dmapreduce.reduce.speculative=false</pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="rowcounter"><a class="anchor" href="#rowcounter"></a>159.17. RowCounter</h3>
<div class="paragraph">
<p><a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html">RowCounter</a> is a mapreduce job to count all the rows of a table.
This is a good utility to use as a sanity check to ensure that HBase can read all the blocks of a table if there are any concerns of metadata inconsistency.
It will run the mapreduce all in a single process but it will run faster if you have a MapReduce cluster in place for it to exploit.
It is possible to limit the time range of data to be scanned by using the <code>--starttime=[starttime]</code> and <code>--endtime=[endtime]</code> flags.
The scanned data can be limited based on keys using the <code>--range=[startKey],[endKey][;[startKey],[endKey]&#8230;&#8203;]</code> option.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase rowcounter [options] &lt;tablename&gt; [--starttime=&lt;start&gt; --endtime=&lt;end&gt;] [--range=[startKey],[endKey][;[startKey],[endKey]...]] [&lt;column1&gt; &lt;column2&gt;...]</pre>
</div>
</div>
<div class="paragraph">
<p>RowCounter only counts one version per cell.</p>
</div>
<div class="paragraph">
<p>For performance consider to use <code>-Dhbase.client.scanner.caching=100</code> and <code>-Dmapreduce.map.speculative=false</code> options.</p>
</div>
</div>
<div class="sect2">
<h3 id="cellcounter"><a class="anchor" href="#cellcounter"></a>159.18. CellCounter</h3>
<div class="paragraph">
<p>HBase ships another diagnostic mapreduce job called <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/CellCounter.html">CellCounter</a>.
Like RowCounter, it gathers more fine-grained statistics about your table.
The statistics gathered by CellCounter are more fine-grained and include:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Total number of rows in the table.</p>
</li>
<li>
<p>Total number of CFs across all rows.</p>
</li>
<li>
<p>Total qualifiers across all rows.</p>
</li>
<li>
<p>Total occurrence of each CF.</p>
</li>
<li>
<p>Total occurrence of each qualifier.</p>
</li>
<li>
<p>Total number of versions of each qualifier.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The program allows you to limit the scope of the run.
Provide a row regex or prefix to limit the rows to analyze.
Specify a time range to scan the table by using the <code>--starttime=&lt;starttime&gt;</code> and <code>--endtime=&lt;endtime&gt;</code> flags.</p>
</div>
<div class="paragraph">
<p>Use <code>hbase.mapreduce.scan.column.family</code> to specify scanning a single column family.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase cellcounter &lt;tablename&gt; &lt;outputDir&gt; [reportSeparator] [regex or prefix] [--starttime=&lt;starttime&gt; --endtime=&lt;endtime&gt;]</pre>
</div>
</div>
<div class="paragraph">
<p>Note: just like RowCounter, caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the job configuration.</p>
</div>
</div>
<div class="sect2">
<h3 id="_mlockall"><a class="anchor" href="#_mlockall"></a>159.19. mlockall</h3>
<div class="paragraph">
<p>It is possible to optionally pin your servers in physical memory making them less likely to be swapped out in oversubscribed environments by having the servers call <a href="http://linux.die.net/man/2/mlockall">mlockall</a> on startup.
See <a href="https://issues.apache.org/jira/browse/HBASE-4391">HBASE-4391 Add ability to start RS as root and call mlockall</a> for how to build the optional library and have it run on startup.</p>
</div>
</div>
<div class="sect2">
<h3 id="compaction.tool"><a class="anchor" href="#compaction.tool"></a>159.20. Offline Compaction Tool</h3>
<div class="paragraph">
<p><strong>CompactionTool</strong> provides a way of running compactions (either minor or major) as an independent
process from the RegionServer. It reuses same internal implementation classes executed by RegionServer
compaction feature. However, since this runs on a complete separate independent java process, it
releases RegionServers from the overhead involved in rewrite a set of hfiles, which can be critical
for latency sensitive use cases.</p>
</div>
<div class="paragraph">
<p>Usage:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase org.apache.hadoop.hbase.regionserver.CompactionTool
Usage: java org.apache.hadoop.hbase.regionserver.CompactionTool \
[-compactOnce] [-major] [-mapred] [-D&lt;property=value&gt;]* files...
Options:
mapred Use MapReduce to run compaction.
compactOnce Execute just one compaction step. (default: while needed)
major Trigger major compaction.
Note: -D properties will be applied to the conf used.
For example:
To stop delete of compacted file, pass -Dhbase.compactiontool.delete=false
To set tmp dir, pass -Dhbase.tmp.dir=ALTERNATE_DIR
Examples:
To compact the full 'TestTable' using MapReduce:
$ hbase org.apache.hadoop.hbase.regionserver.CompactionTool -mapred hdfs://hbase/data/default/TestTable
To compact column family 'x' of the table 'TestTable' region 'abc':
$ hbase org.apache.hadoop.hbase.regionserver.CompactionTool hdfs://hbase/data/default/TestTable/abc/x</pre>
</div>
</div>
<div class="paragraph">
<p>As shown by usage options above, <strong>CompactionTool</strong> can run as a standalone client or a mapreduce job.
When running as mapreduce job, each family dir is handled as an input split, and is processed
by a separate map task.</p>
</div>
<div class="paragraph">
<p>The <strong>compactionOnce</strong> parameter controls how many compaction cycles will be performed until
<strong>CompactionTool</strong> program decides to finish its work. If omitted, it will assume it should keep
running compactions on each specified family as determined by the given compaction policy
configured. For more info on compaction policy, see <a href="#compaction">compaction</a>.</p>
</div>
<div class="paragraph">
<p>If a major compaction is desired, <strong>major</strong> flag can be specified. If omitted, <strong>CompactionTool</strong> will
assume minor compaction is wanted by default.</p>
</div>
<div class="paragraph">
<p>It also allows for configuration overrides with <code>-D</code> flag. In the usage section above, for example,
<code>-Dhbase.compactiontool.delete=false</code> option will instruct compaction engine to not delete original
files from temp folder.</p>
</div>
<div class="paragraph">
<p>Files targeted for compaction must be specified as parent hdfs dirs. It allows for multiple dirs
definition, as long as each for these dirs are either a <strong>family</strong>, a <strong>region</strong>, or a <strong>table</strong> dir. If a
table or region dir is passed, the program will recursively iterate through related sub-folders,
effectively running compaction for each family found below the table/region level.</p>
</div>
<div class="paragraph">
<p>Since these dirs are nested under <strong>hbase</strong> hdfs directory tree, <strong>CompactionTool</strong> requires hbase super
user permissions in order to have access to required hfiles.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Running in MapReduce mode</div>
<div class="paragraph">
<p>MapReduce mode offers the ability to process each family dir in parallel, as a separate map task.
Generally, it would make sense to run in this mode when specifying one or more table dirs as targets
for compactions. The caveat, though, is that if number of families to be compacted become too large,
the related mapreduce job may have indirect impacts on <strong>RegionServers</strong> performance .
Since <strong>NodeManagers</strong> are normally co-located with RegionServers, such large jobs could
compete for IO/Bandwidth resources with the <strong>RegionServers</strong>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">MajorCompaction completely disabled on RegionServers due performance impacts</div>
<div class="paragraph">
<p><strong>Major compactions</strong> can be a costly operation (see <a href="#compaction">compaction</a>), and can indeed
impact performance on RegionServers, leading operators to completely disable it for critical
low latency application. <strong>CompactionTool</strong> could be used as an alternative in such scenarios,
although, additional custom application logic would need to be implemented, such as deciding
scheduling and selection of tables/regions/families target for a given compaction run.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>For additional details about CompactionTool, see also
<a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html">CompactionTool</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="__code_hbase_clean_code"><a class="anchor" href="#__code_hbase_clean_code"></a>159.21. <code>hbase clean</code></h3>
<div class="paragraph">
<p>The <code>hbase clean</code> command cleans HBase data from ZooKeeper, HDFS, or both.
It is appropriate to use for testing.
Run it with no options for usage instructions.
The <code>hbase clean</code> command was introduced in HBase 0.98.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase clean
Usage: hbase clean (--cleanZk|--cleanHdfs|--cleanAll)
Options:
--cleanZk cleans hbase related data from zookeeper.
--cleanHdfs cleans hbase related data from hdfs.
--cleanAll cleans hbase related data from both zookeeper and hdfs.</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="__code_hbase_pe_code"><a class="anchor" href="#__code_hbase_pe_code"></a>159.22. <code>hbase pe</code></h3>
<div class="paragraph">
<p>The <code>hbase pe</code> command runs the PerformanceEvaluation tool, which is used for testing.</p>
</div>
<div class="paragraph">
<p>The PerformanceEvaluation tool accepts many different options and commands.
For usage instructions, run the command with no options.</p>
</div>
<div class="paragraph">
<p>The PerformanceEvaluation tool has received many updates in recent HBase releases, including support for namespaces, support for tags, cell-level ACLs and visibility labels, multiget support for RPC calls, increased sampling sizes, an option to randomly sleep during testing, and ability to "warm up" the cluster before testing starts.</p>
</div>
</div>
<div class="sect2">
<h3 id="__code_hbase_ltt_code"><a class="anchor" href="#__code_hbase_ltt_code"></a>159.23. <code>hbase ltt</code></h3>
<div class="paragraph">
<p>The <code>hbase ltt</code> command runs the LoadTestTool utility, which is used for testing.</p>
</div>
<div class="paragraph">
<p>You must specify either <code>-init_only</code> or at least one of <code>-write</code>, <code>-update</code>, or <code>-read</code>.
For general usage instructions, pass the <code>-h</code> option.</p>
</div>
<div class="paragraph">
<p>The LoadTestTool has received many updates in recent HBase releases, including support for namespaces, support for tags, cell-level ACLS and visibility labels, testing security-related features, ability to specify the number of regions per server, tests for multi-get RPC calls, and tests relating to replication.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.pre-upgrade"><a class="anchor" href="#ops.pre-upgrade"></a>159.24. Pre-Upgrade validator</h3>
<div class="paragraph">
<p>Pre-Upgrade validator tool can be used to check the cluster for known incompatibilities before upgrading from HBase 1 to HBase 2.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade command ...</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="_coprocessor_validation"><a class="anchor" href="#_coprocessor_validation"></a>159.24.1. Coprocessor validation</h4>
<div class="paragraph">
<p>HBase supports co-processors for a long time, but the co-processor API can be changed between major releases. Co-processor validator tries to determine
whether the old co-processors are still compatible with the actual HBase version.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade validate-cp [-jar ...] [-class ... | -table ... | -config]
Options:
-e Treat warnings as errors.
-jar &lt;arg&gt; Jar file/directory of the coprocessor.
-table &lt;arg&gt; Table coprocessor(s) to check.
-class &lt;arg&gt; Coprocessor class(es) to check.
-config Scan jar for observers.</code></pre>
</div>
</div>
<div class="paragraph">
<p>The co-processor classes can be explicitly declared by <code>-class</code> option, or they can be obtained from HBase configuration by <code>-config</code> option.
Table level co-processors can be also checked by <code>-table</code> option. The tool searches for co-processors on its classpath, but it can be extended
by the <code>-jar</code> option. It is possible to test multiple classes with multiple <code>-class</code>, multiple tables with multiple <code>-table</code> options as well as
adding multiple jars to the classpath with multiple <code>-jar</code> options.</p>
</div>
<div class="paragraph">
<p>The tool can report errors and warnings. Errors mean that HBase won&#8217;t be able to load the coprocessor, because it is incompatible with the current version
of HBase. Warnings mean that the co-processors can be loaded, but they won&#8217;t work as expected. If <code>-e</code> option is given, then the tool will also fail
for warnings.</p>
</div>
<div class="paragraph">
<p>Please note that this tool cannot validate every aspect of jar files, it just does some static checks.</p>
</div>
<div class="paragraph">
<p>For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade validate-cp -jar my-coprocessor.jar -class MyMasterObserver -class MyRegionObserver</code></pre>
</div>
</div>
<div class="paragraph">
<p>It validates <code>MyMasterObserver</code> and <code>MyRegionObserver</code> classes which are located in <code>my-coprocessor.jar</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade validate-cp -table .*</code></pre>
</div>
</div>
<div class="paragraph">
<p>It validates every table level co-processors where the table name matches to <code>.*</code> regular expression.</p>
</div>
</div>
<div class="sect3">
<h4 id="_datablockencoding_validation"><a class="anchor" href="#_datablockencoding_validation"></a>159.24.2. DataBlockEncoding validation</h4>
<div class="paragraph">
<p>HBase 2.0 removed <code>PREFIX_TREE</code> Data Block Encoding from column families. For further information
please check <a href="#upgrade2.0.prefix-tree.removed"><em>prefix-tree</em> encoding removed</a>.
To verify that none of the column families are using incompatible Data Block Encodings in the cluster run the following command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade validate-dbe</code></pre>
</div>
</div>
<div class="paragraph">
<p>This check validates all column families and print out any incompatibilities. For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-07-13 09:58:32,028 WARN [main] tool.DataBlockEncodingValidator: Incompatible DataBlockEncoding for table: t, cf: f, encoding: PREFIX_TREE</pre>
</div>
</div>
<div class="paragraph">
<p>Which means that Data Block Encoding of table <code>t</code>, column family <code>f</code> is incompatible. To fix, use <code>alter</code> command in HBase shell:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>alter 't', { NAME =&gt; 'f', DATA_BLOCK_ENCODING =&gt; 'FAST_DIFF' }</pre>
</div>
</div>
<div class="paragraph">
<p>Please also validate HFiles, which is described in the next section.</p>
</div>
</div>
<div class="sect3">
<h4 id="_hfile_content_validation"><a class="anchor" href="#_hfile_content_validation"></a>159.24.3. HFile Content validation</h4>
<div class="paragraph">
<p>Even though Data Block Encoding is changed from <code>PREFIX_TREE</code> it is still possible to have HFiles that contain data encoded that way.
To verify that HFiles are readable with HBase 2 please use <em>HFile content validator</em>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase pre-upgrade validate-hfile</code></pre>
</div>
</div>
<div class="paragraph">
<p>The tool will log the corrupt HFiles and details about the root cause.
If the problem is about PREFIX_TREE encoding it is necessary to change encodings before upgrading to HBase 2.</p>
</div>
<div class="paragraph">
<p>The following log message shows an example of incorrect HFiles.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2018-06-05 16:20:46,976 WARN [hfilevalidator-pool1-t3] hbck.HFileCorruptionChecker: Found corrupt HFile hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
org.apache.hadoop.hbase.io.hfile.CorruptHFileException: Problem reading HFile Trailer from file hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
...
Caused by: java.io.IOException: Invalid data block encoding type in file info: PREFIX_TREE
...
Caused by: java.lang.IllegalArgumentException: No enum constant org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.PREFIX_TREE
...
2018-06-05 16:20:47,322 INFO [main] tool.HFileContentValidator: Corrupted file: hdfs://example.com:8020/hbase/data/default/t/72ea7f7d625ee30f959897d1a3e2c350/prefix/7e6b3d73263c4851bf2b8590a9b3791e
2018-06-05 16:20:47,383 INFO [main] tool.HFileContentValidator: Corrupted file: hdfs://example.com:8020/hbase/archive/data/default/t/56be41796340b757eb7fff1eb5e2a905/f/29c641ae91c34fc3bee881f45436b6d1</pre>
</div>
</div>
<div class="sect4">
<h5 id="_fixing_prefix_tree_errors"><a class="anchor" href="#_fixing_prefix_tree_errors"></a>Fixing PREFIX_TREE errors</h5>
<div class="paragraph">
<p>It&#8217;s possible to get <code>PREFIX_TREE</code> errors after changing Data Block Encoding to a supported one. It can happen
because there are some HFiles which still encoded with <code>PREFIX_TREE</code> or there are still some snapshots.</p>
</div>
<div class="paragraph">
<p>For fixing HFiles, please run a major compaction on the table (it was <code>default:t</code> according to the log message):</p>
</div>
<div class="listingblock">
<div class="content">
<pre>major_compact 't'</pre>
</div>
</div>
<div class="paragraph">
<p>HFiles can be referenced from snapshots, too. It&#8217;s the case when the HFile is located under <code>archive/data</code>.
The first step is to determine which snapshot references that HFile (the name of the file was <code>29c641ae91c34fc3bee881f45436b6d1</code>
according to the logs):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">for snapshot in $(hbase snapshotinfo -list-snapshots 2&gt; /dev/null | tail -n -1 | cut -f 1 -d \|);
do
echo &quot;checking snapshot named '${snapshot}'&quot;;
hbase snapshotinfo -snapshot &quot;${snapshot}&quot; -files 2&gt; /dev/null | grep 29c641ae91c34fc3bee881f45436b6d1;
done</code></pre>
</div>
</div>
<div class="paragraph">
<p>The output of this shell script is:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>checking snapshot named 't_snap'
1.0 K t/56be41796340b757eb7fff1eb5e2a905/f/29c641ae91c34fc3bee881f45436b6d1 (archive)</pre>
</div>
</div>
<div class="paragraph">
<p>Which means <code>t_snap</code> snapshot references the incompatible HFile. If the snapshot is still needed,
then it has to be recreated with HBase shell:</p>
</div>
<div class="listingblock">
<div class="content">
<pre># creating a new namespace for the cleanup process
create_namespace 'pre_upgrade_cleanup'
# creating a new snapshot
clone_snapshot 't_snap', 'pre_upgrade_cleanup:t'
alter 'pre_upgrade_cleanup:t', { NAME =&gt; 'f', DATA_BLOCK_ENCODING =&gt; 'FAST_DIFF' }
major_compact 'pre_upgrade_cleanup:t'
# removing the invalid snapshot
delete_snapshot 't_snap'
# creating a new snapshot
snapshot 'pre_upgrade_cleanup:t', 't_snap'
# removing temporary table
disable 'pre_upgrade_cleanup:t'
drop 'pre_upgrade_cleanup:t'
drop_namespace 'pre_upgrade_cleanup'</pre>
</div>
</div>
<div class="paragraph">
<p>For further information, please refer to
<a href="https://issues.apache.org/jira/browse/HBASE-20649?focusedCommentId=16535476#comment-16535476">HBASE-20649</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_data_block_encoding_tool"><a class="anchor" href="#_data_block_encoding_tool"></a>159.25. Data Block Encoding Tool</h3>
<div class="paragraph">
<p>Tests various compression algorithms with different data block encoder for key compression on an existing HFile.
Useful for testing, debugging and benchmarking.</p>
</div>
<div class="paragraph">
<p>You must specify <code>-f</code> which is the full path of the HFile.</p>
</div>
<div class="paragraph">
<p>The result shows both the performance (MB/s) of compression/decompression and encoding/decoding, and the data savings on the HFile.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.regionserver.DataBlockEncodingTool
Usages: hbase org.apache.hadoop.hbase.regionserver.DataBlockEncodingTool
Options:
-f HFile to analyse (REQUIRED)
-n Maximum number of key/value pairs to process in a single benchmark run.
-b Whether to run a benchmark to measure read throughput.
-c If this is specified, no correctness testing will be done.
-a What kind of compression algorithm use for test. Default value: GZ.
-t Number of times to run each benchmark. Default value: 12.
-omit Number of first runs of every benchmark to omit from statistics. Default value: 2.</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ops.regionmgt"><a class="anchor" href="#ops.regionmgt"></a>160. Region Management</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="ops.regionmgt.majorcompact"><a class="anchor" href="#ops.regionmgt.majorcompact"></a>160.1. Major Compaction</h3>
<div class="paragraph">
<p>Major compactions can be requested via the HBase shell or <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Admin.html#majorCompact-org.apache.hadoop.hbase.TableName-">Admin.majorCompact</a>.</p>
</div>
<div class="paragraph">
<p>Note: major compactions do NOT do region merges.
See <a href="#compaction">compaction</a> for more information about compactions.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.regionmgt.merge"><a class="anchor" href="#ops.regionmgt.merge"></a>160.2. Merge</h3>
<div class="paragraph">
<p>Merge is a utility that can merge adjoining regions in the same table (see org.apache.hadoop.hbase.util.Merge).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ bin/hbase org.apache.hadoop.hbase.util.Merge &lt;tablename&gt; &lt;region1&gt; &lt;region2&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you feel you have too many regions and want to consolidate them, Merge is the utility you need.
Merge must run be done when the cluster is down.
See the <a href="https://web.archive.org/web/20111231002503/http://ofps.oreilly.com/titles/9781449396107/performance.html">O&#8217;Reilly HBase
Book</a> for an example of usage.</p>
</div>
<div class="paragraph">
<p>You will need to pass 3 parameters to this application.
The first one is the table name.
The second one is the fully qualified name of the first region to merge, like "table_name,\x0A,1342956111995.7cef47f192318ba7ccc75b1bbf27a82b.". The third one is the fully qualified name for the second region to merge.</p>
</div>
<div class="paragraph">
<p>Additionally, there is a Ruby script attached to <a href="https://issues.apache.org/jira/browse/HBASE-1621">HBASE-1621</a> for region merging.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="node.management"><a class="anchor" href="#node.management"></a>161. Node Management</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="decommission"><a class="anchor" href="#decommission"></a>161.1. Node Decommission</h3>
<div class="paragraph">
<p>You can stop an individual RegionServer by running the following script in the HBase directory on the particular node:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase-daemon.sh stop regionserver</pre>
</div>
</div>
<div class="paragraph">
<p>The RegionServer will first close all regions and then shut itself down.
On shutdown, the RegionServer&#8217;s ephemeral node in ZooKeeper will expire.
The master will notice the RegionServer gone and will treat it as a 'crashed' server; it will reassign the nodes the RegionServer was carrying.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Disable the Load Balancer before Decommissioning a node</div>
<div class="paragraph">
<p>If the load balancer runs while a node is shutting down, then there could be contention between the Load Balancer and the Master&#8217;s recovery of the just decommissioned RegionServer.
Avoid any problems by disabling the balancer first.
See <a href="#lb">lb</a> below.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Kill Node Tool</div>
<div class="paragraph">
<p>In hbase-2.0, in the bin directory, we added a script named <em>considerAsDead.sh</em> that can be used to kill a regionserver.
Hardware issues could be detected by specialized monitoring tools before the zookeeper timeout has expired. <em>considerAsDead.sh</em> is a simple function to mark a RegionServer as dead.
It deletes all the znodes of the server, starting the recovery process.
Plug in the script into your monitoring/fault detection tools to initiate faster failover.
Be careful how you use this disruptive tool.
Copy the script if you need to make use of it in a version of hbase previous to hbase-2.0.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>A downside to the above stop of a RegionServer is that regions could be offline for a good period of time.
Regions are closed in order.
If many regions on the server, the first region to close may not be back online until all regions close and after the master notices the RegionServer&#8217;s znode gone.
In Apache HBase 0.90.2, we added facility for having a node gradually shed its load and then shutdown itself down.
Apache HBase 0.90.2 added the <em>graceful_stop.sh</em> script.
Here is its usage:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/graceful_stop.sh
Usage: graceful_stop.sh [--config &amp;conf-dir&gt;] [--restart] [--reload] [--thrift] [--rest] &amp;hostname&gt;
thrift If we should stop/start thrift before/after the hbase stop/start
rest If we should stop/start rest before/after the hbase stop/start
restart If we should restart after graceful stop
reload Move offloaded regions back on to the stopped server
debug Move offloaded regions back on to the stopped server
hostname Hostname of server we are to stop</pre>
</div>
</div>
<div class="paragraph">
<p>To decommission a loaded RegionServer, run the following: $
./bin/graceful_stop.sh HOSTNAME where <code>HOSTNAME</code> is the host carrying the RegionServer you would decommission.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">On <code>HOSTNAME</code></div>
<div class="paragraph">
<p>The <code>HOSTNAME</code> passed to <em>graceful_stop.sh</em> must match the hostname that hbase is using to identify RegionServers.
Check the list of RegionServers in the master UI for how HBase is referring to servers.
It&#8217;s usually hostname but can also be FQDN.
Whatever HBase is using, this is what you should pass the <em>graceful_stop.sh</em> decommission script.
If you pass IPs, the script is not yet smart enough to make a hostname (or FQDN) of it and so it will fail when it checks if server is currently running; the graceful unloading of regions will not run.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The <em>graceful_stop.sh</em> script will move the regions off the decommissioned RegionServer one at a time to minimize region churn.
It will verify the region deployed in the new location before it will moves the next region and so on until the decommissioned server is carrying zero regions.
At this point, the <em>graceful_stop.sh</em> tells the RegionServer <code>stop</code>.
The master will at this point notice the RegionServer gone but all regions will have already been redeployed and because the RegionServer went down cleanly, there will be no WAL logs to split.</p>
</div>
<div id="lb" class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Load Balancer</div>
<div class="paragraph">
<p>It is assumed that the Region Load Balancer is disabled while the <code>graceful_stop</code> script runs (otherwise the balancer and the decommission script will end up fighting over region deployments). Use the shell to disable the balancer:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; balance_switch <span class="predefined-constant">false</span>
<span class="predefined-constant">true</span>
<span class="integer">0</span> row(s) in <span class="float">0.3590</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>This turns the balancer OFF.
To reenable, do:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; balance_switch <span class="predefined-constant">true</span>
<span class="predefined-constant">false</span>
<span class="integer">0</span> row(s) in <span class="float">0.3590</span> seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>The <code>graceful_stop</code> will check the balancer and if enabled, will turn it off before it goes to work.
If it exits prematurely because of error, it will not have reset the balancer.
Hence, it is better to manage the balancer apart from <code>graceful_stop</code> reenabling it after you are done w/ graceful_stop.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="draining.servers"><a class="anchor" href="#draining.servers"></a>161.1.1. Decommissioning several Regions Servers concurrently</h4>
<div class="paragraph">
<p>If you have a large cluster, you may want to decommission more than one machine at a time by gracefully stopping multiple RegionServers concurrently.
To gracefully drain multiple regionservers at the same time, RegionServers can be put into a "draining" state.
This is done by marking a RegionServer as a draining node by creating an entry in ZooKeeper under the <em>hbase_root/draining</em> znode.
This znode has format <code>name,port,startcode</code> just like the regionserver entries under <em>hbase_root/rs</em> znode.</p>
</div>
<div class="paragraph">
<p>Without this facility, decommissioning multiple nodes may be non-optimal because regions that are being drained from one region server may be moved to other regionservers that are also draining.
Marking RegionServers to be in the draining state prevents this from happening.
See this <a href="http://inchoate-clatter.blogspot.com/2012/03/hbase-ops-automation.html">blog
post</a> for more details.</p>
</div>
</div>
<div class="sect3">
<h4 id="bad.disk"><a class="anchor" href="#bad.disk"></a>161.1.2. Bad or Failing Disk</h4>
<div class="paragraph">
<p>It is good having <a href="#dfs.datanode.failed.volumes.tolerated">dfs.datanode.failed.volumes.tolerated</a> set if you have a decent number of disks per machine for the case where a disk plain dies.
But usually disks do the "John Wayne"&#8201;&#8212;&#8201;i.e.
take a while to go down spewing errors in <em>dmesg</em>&#8201;&#8212;&#8201;or for some reason, run much slower than their companions.
In this case you want to decommission the disk.
You have two options.
You can <a href="https://wiki.apache.org/hadoop/FAQ#I_want_to_make_a_large_cluster_smaller_by_taking_out_a_bunch_of_nodes_simultaneously._How_can_this_be_done.3F">decommission
the datanode</a> or, less disruptive in that only the bad disks data will be rereplicated, can stop the datanode, unmount the bad volume (You can&#8217;t umount a volume while the datanode is using it), and then restart the datanode (presuming you have set dfs.datanode.failed.volumes.tolerated &gt; 0). The regionserver will throw some errors in its logs as it recalibrates where to get its data from&#8201;&#8212;&#8201;it will likely roll its WAL log too&#8201;&#8212;&#8201;but in general but for some latency spikes, it should keep on chugging.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Short Circuit Reads</div>
<div class="paragraph">
<p>If you are doing short-circuit reads, you will have to move the regions off the regionserver before you stop the datanode; when short-circuiting reading, though chmod&#8217;d so regionserver cannot have access, because it already has the files open, it will be able to keep reading the file blocks from the bad disk even though the datanode is down.
Move the regions back after you restart the datanode.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect2">
<h3 id="rolling"><a class="anchor" href="#rolling"></a>161.2. Rolling Restart</h3>
<div class="paragraph">
<p>Some cluster configuration changes require either the entire cluster, or the RegionServers, to be restarted in order to pick up the changes.
In addition, rolling restarts are supported for upgrading to a minor or maintenance release, and to a major release if at all possible.
See the release notes for release you want to upgrade to, to find out about limitations to the ability to perform a rolling upgrade.</p>
</div>
<div class="paragraph">
<p>There are multiple ways to restart your cluster nodes, depending on your situation.
These methods are detailed below.</p>
</div>
<div class="sect3">
<h4 id="_using_the_code_rolling_restart_sh_code_script"><a class="anchor" href="#_using_the_code_rolling_restart_sh_code_script"></a>161.2.1. Using the <code>rolling-restart.sh</code> Script</h4>
<div class="paragraph">
<p>HBase ships with a script, <em>bin/rolling-restart.sh</em>, that allows you to perform rolling restarts on the entire cluster, the master only, or the RegionServers only.
The script is provided as a template for your own script, and is not explicitly tested.
It requires password-less SSH login to be configured and assumes that you have deployed using a tarball.
The script requires you to set some environment variables before running it.
Examine the script and modify it to suit your needs.</p>
</div>
<div class="listingblock">
<div class="title"><em>rolling-restart.sh</em> General Usage</div>
<div class="content">
<pre>$ ./bin/rolling-restart.sh --help
Usage: rolling-restart.sh [--config &lt;hbase-confdir&gt;] [--rs-only] [--master-only] [--graceful] [--maxthreads xx]</pre>
</div>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Rolling Restart on RegionServers Only</dt>
<dd>
<p>To perform a rolling restart on the RegionServers only, use the <code>--rs-only</code> option.
This might be necessary if you need to reboot the individual RegionServer or if you make a configuration change that only affects RegionServers and not the other HBase processes.</p>
</dd>
<dt class="hdlist1">Rolling Restart on Masters Only</dt>
<dd>
<p>To perform a rolling restart on the active and backup Masters, use the <code>--master-only</code> option.
You might use this if you know that your configuration change only affects the Master and not the RegionServers, or if you need to restart the server where the active Master is running.</p>
</dd>
<dt class="hdlist1">Graceful Restart</dt>
<dd>
<p>If you specify the <code>--graceful</code> option, RegionServers are restarted using the <em>bin/graceful_stop.sh</em> script, which moves regions off a RegionServer before restarting it.
This is safer, but can delay the restart.</p>
</dd>
<dt class="hdlist1">Limiting the Number of Threads</dt>
<dd>
<p>To limit the rolling restart to using only a specific number of threads, use the <code>--maxthreads</code> option.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="rolling.restart.manual"><a class="anchor" href="#rolling.restart.manual"></a>161.2.2. Manual Rolling Restart</h4>
<div class="paragraph">
<p>To retain more control over the process, you may wish to manually do a rolling restart across your cluster.
This uses the <code>graceful-stop.sh</code> command <a href="#decommission">decommission</a>.
In this method, you can restart each RegionServer individually and then move its old regions back into place, retaining locality.
If you also need to restart the Master, you need to do it separately, and restart the Master before restarting the RegionServers using this method.
The following is an example of such a command.
You may need to tailor it to your environment.
This script does a rolling restart of RegionServers only.
It disables the load balancer before moving the regions.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i; done &amp;&gt; /tmp/log.txt &amp;;</pre>
</div>
</div>
<div class="paragraph">
<p>Monitor the output of the <em>/tmp/log.txt</em> file to follow the progress of the script.</p>
</div>
</div>
<div class="sect3">
<h4 id="_logic_for_crafting_your_own_rolling_restart_script"><a class="anchor" href="#_logic_for_crafting_your_own_rolling_restart_script"></a>161.2.3. Logic for Crafting Your Own Rolling Restart Script</h4>
<div class="paragraph">
<p>Use the following guidelines if you want to create your own rolling restart script.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Extract the new release, verify its configuration, and synchronize it to all nodes of your cluster using <code>rsync</code>, <code>scp</code>, or another secure synchronization mechanism.</p>
</li>
<li>
<p>Restart the master first.
You may need to modify these commands if your new HBase directory is different from the old one, such as for an upgrade.</p>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase-daemon.sh stop master; ./bin/hbase-daemon.sh start master</pre>
</div>
</div>
</li>
<li>
<p>Gracefully restart each RegionServer, using a script such as the following, from the Master.</p>
<div class="listingblock">
<div class="content">
<pre>$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i; done &amp;&gt; /tmp/log.txt &amp;</pre>
</div>
</div>
<div class="paragraph">
<p>If you are running Thrift or REST servers, pass the --thrift or --rest options.
For other available options, run the <code>bin/graceful-stop.sh --help</code> command.</p>
</div>
<div class="paragraph">
<p>It is important to drain HBase regions slowly when restarting multiple RegionServers.
Otherwise, multiple regions go offline simultaneously and must be reassigned to other nodes, which may also go offline soon.
This can negatively affect performance.
You can inject delays into the script above, for instance, by adding a Shell command such as <code>sleep</code>.
To wait for 5 minutes between each RegionServer restart, modify the above script to the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ for i in `cat conf/regionservers|sort`; do ./bin/graceful_stop.sh --restart --reload --debug $i &amp; sleep 5m; done &amp;&gt; /tmp/log.txt &amp;</pre>
</div>
</div>
</li>
<li>
<p>Restart the Master again, to clear out the dead servers list and re-enable the load balancer.</p>
</li>
</ol>
</div>
</div>
</div>
<div class="sect2">
<h3 id="adding.new.node"><a class="anchor" href="#adding.new.node"></a>161.3. Adding a New Node</h3>
<div class="paragraph">
<p>Adding a new regionserver in HBase is essentially free, you simply start it like this: <code>$ ./bin/hbase-daemon.sh start regionserver</code> and it will register itself with the master.
Ideally you also started a DataNode on the same machine so that the RS can eventually start to have local files.
If you rely on ssh to start your daemons, don&#8217;t forget to add the new hostname in <em>conf/regionservers</em> on the master.</p>
</div>
<div class="paragraph">
<p>At this point the region server isn&#8217;t serving data because no regions have moved to it yet.
If the balancer is enabled, it will start moving regions to the new RS.
On a small/medium cluster this can have a very adverse effect on latency as a lot of regions will be offline at the same time.
It is thus recommended to disable the balancer the same way it&#8217;s done when decommissioning a node and move the regions manually (or even better, using a script that moves them one by one).</p>
</div>
<div class="paragraph">
<p>The moved regions will all have 0% locality and won&#8217;t have any blocks in cache so the region server will have to use the network to serve requests.
Apart from resulting in higher latency, it may also be able to use all of your network card&#8217;s capacity.
For practical purposes, consider that a standard 1GigE NIC won&#8217;t be able to read much more than <em>100MB/s</em>.
In this case, or if you are in a OLAP environment and require having locality, then it is recommended to major compact the moved regions.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase_metrics"><a class="anchor" href="#hbase_metrics"></a>162. HBase Metrics</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase emits metrics which adhere to the <a href="https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/Metrics.html">Hadoop Metrics</a> API.
Starting with HBase 0.95<sup class="footnote">[<a id="_footnoteref_5" class="footnote" href="#_footnote_5" title="View footnote.">5</a>]</sup>, HBase is configured to emit a default set of metrics with a default sampling period of every 10 seconds.
You can use HBase metrics in conjunction with Ganglia.
You can also filter which metrics are emitted and extend the metrics framework to capture custom metrics appropriate for your environment.</p>
</div>
<div class="sect2">
<h3 id="_metric_setup"><a class="anchor" href="#_metric_setup"></a>162.1. Metric Setup</h3>
<div class="paragraph">
<p>For HBase 0.95 and newer, HBase ships with a default metrics configuration, or <em class="firstterm">sink</em>.
This includes a wide variety of individual metrics, and emits them every 10 seconds by default.
To configure metrics for a given region server, edit the <em>conf/hadoop-metrics2-hbase.properties</em> file.
Restart the region server for the changes to take effect.</p>
</div>
<div class="paragraph">
<p>To change the sampling rate for the default sink, edit the line beginning with <code>*.period</code>.
To filter which metrics are emitted or to extend the metrics framework, see <a href="https://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html" class="bare">https://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html</a></p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">HBase Metrics and Ganglia</div>
<div class="paragraph">
<p>By default, HBase emits a large number of metrics per region server.
Ganglia may have difficulty processing all these metrics.
Consider increasing the capacity of the Ganglia server or reducing the number of metrics emitted by HBase.
See <a href="https://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html#filtering">Metrics Filtering</a>.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
<div class="sect2">
<h3 id="_disabling_metrics"><a class="anchor" href="#_disabling_metrics"></a>162.2. Disabling Metrics</h3>
<div class="paragraph">
<p>To disable metrics for a region server, edit the <em>conf/hadoop-metrics2-hbase.properties</em> file and comment out any uncommented lines.
Restart the region server for the changes to take effect.</p>
</div>
</div>
<div class="sect2">
<h3 id="discovering.available.metrics"><a class="anchor" href="#discovering.available.metrics"></a>162.3. Discovering Available Metrics</h3>
<div class="paragraph">
<p>Rather than listing each metric which HBase emits by default, you can browse through the available metrics, either as a JSON output or via JMX.
Different metrics are exposed for the Master process and each region server process.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Access a JSON Output of Available Metrics</div>
<ol class="arabic">
<li>
<p>After starting HBase, access the region server&#8217;s web UI, at http://REGIONSERVER_HOSTNAME:60030 by default (or port 16030 in HBase 1.0+).</p>
</li>
<li>
<p>Click the <span class="label">Metrics Dump</span> link near the top.
The metrics for the region server are presented as a dump of the JMX bean in JSON format.
This will dump out all metrics names and their values.
To include metrics descriptions in the listing&#8201;&#8212;&#8201;this can be useful when you are exploring what is available&#8201;&#8212;&#8201;add a query string of <code>?description=true</code> so your URL becomes http://REGIONSERVER_HOSTNAME:60030/jmx?description=true.
Not all beans and attributes have descriptions.</p>
</li>
<li>
<p>To view metrics for the Master, connect to the Master&#8217;s web UI instead (defaults to http://localhost:60010 or port 16010 in HBase 1.0+) and click its <span class="label">Metrics
Dump</span> link.
To include metrics descriptions in the listing&#8201;&#8212;&#8201;this can be useful when you are exploring what is available&#8201;&#8212;&#8201;add a query string of <code>?description=true</code> so your URL becomes http://REGIONSERVER_HOSTNAME:60010/jmx?description=true.
Not all beans and attributes have descriptions.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>You can use many different tools to view JMX content by browsing MBeans.
This procedure uses <code>jvisualvm</code>, which is an application usually available in the JDK.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Browse the JMX Output of Available Metrics</div>
<ol class="arabic">
<li>
<p>Start HBase, if it is not already running.</p>
</li>
<li>
<p>Run the command <code>jvisualvm</code> command on a host with a GUI display.
You can launch it from the command line or another method appropriate for your operating system.</p>
</li>
<li>
<p>Be sure the <span class="label">VisualVM-MBeans</span> plugin is installed. Browse to <strong>Tools &#8594; Plugins</strong>. Click <span class="label">Installed</span> and check whether the plugin is listed.
If not, click <span class="label">Available Plugins</span>, select it, and click <b class="button">Install</b>.
When finished, click <b class="button">Close</b>.</p>
</li>
<li>
<p>To view details for a given HBase process, double-click the process in the <span class="label">Local</span> sub-tree in the left-hand panel.
A detailed view opens in the right-hand panel.
Click the <span class="label">MBeans</span> tab which appears as a tab in the top of the right-hand panel.</p>
</li>
<li>
<p>To access the HBase metrics, navigate to the appropriate sub-bean:
.* Master:
.* RegionServer:</p>
</li>
<li>
<p>The name of each metric and its current value is displayed in the <span class="label">Attributes</span> tab.
For a view which includes more details, including the description of each attribute, click the <span class="label">Metadata</span> tab.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_units_of_measure_for_metrics"><a class="anchor" href="#_units_of_measure_for_metrics"></a>162.4. Units of Measure for Metrics</h3>
<div class="paragraph">
<p>Different metrics are expressed in different units, as appropriate.
Often, the unit of measure is in the name (as in the metric <code>shippedKBs</code>). Otherwise, use the following guidelines.
When in doubt, you may need to examine the source for a given metric.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Metrics that refer to a point in time are usually expressed as a timestamp.</p>
</li>
<li>
<p>Metrics that refer to an age (such as <code>ageOfLastShippedOp</code>) are usually expressed in milliseconds.</p>
</li>
<li>
<p>Metrics that refer to memory sizes are in bytes.</p>
</li>
<li>
<p>Sizes of queues (such as <code>sizeOfLogQueue</code>) are expressed as the number of items in the queue.
Determine the size by multiplying by the block size (default is 64 MB in HDFS).</p>
</li>
<li>
<p>Metrics that refer to things like the number of a given type of operations (such as <code>logEditsRead</code>) are expressed as an integer.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="master_metrics"><a class="anchor" href="#master_metrics"></a>162.5. Most Important Master Metrics</h3>
<div class="paragraph">
<p>Note: Counts are usually over the last metrics reporting interval.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">hbase.master.numRegionServers</dt>
<dd>
<p>Number of live regionservers</p>
</dd>
<dt class="hdlist1">hbase.master.numDeadRegionServers</dt>
<dd>
<p>Number of dead regionservers</p>
</dd>
<dt class="hdlist1">hbase.master.ritCount </dt>
<dd>
<p>The number of regions in transition</p>
</dd>
<dt class="hdlist1">hbase.master.ritCountOverThreshold</dt>
<dd>
<p>The number of regions that have been in transition longer than a threshold time (default: 60 seconds)</p>
</dd>
<dt class="hdlist1">hbase.master.ritOldestAge</dt>
<dd>
<p>The age of the longest region in transition, in milliseconds</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="rs_metrics"><a class="anchor" href="#rs_metrics"></a>162.6. Most Important RegionServer Metrics</h3>
<div class="paragraph">
<p>Note: Counts are usually over the last metrics reporting interval.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">hbase.regionserver.regionCount</dt>
<dd>
<p>The number of regions hosted by the regionserver</p>
</dd>
<dt class="hdlist1">hbase.regionserver.storeFileCount</dt>
<dd>
<p>The number of store files on disk currently managed by the regionserver</p>
</dd>
<dt class="hdlist1">hbase.regionserver.storeFileSize</dt>
<dd>
<p>Aggregate size of the store files on disk</p>
</dd>
<dt class="hdlist1">hbase.regionserver.hlogFileCount</dt>
<dd>
<p>The number of write ahead logs not yet archived</p>
</dd>
<dt class="hdlist1">hbase.regionserver.totalRequestCount</dt>
<dd>
<p>The total number of requests received</p>
</dd>
<dt class="hdlist1">hbase.regionserver.readRequestCount</dt>
<dd>
<p>The number of read requests received</p>
</dd>
<dt class="hdlist1">hbase.regionserver.writeRequestCount</dt>
<dd>
<p>The number of write requests received</p>
</dd>
<dt class="hdlist1">hbase.regionserver.numOpenConnections</dt>
<dd>
<p>The number of open connections at the RPC layer</p>
</dd>
<dt class="hdlist1">hbase.regionserver.numActiveHandler</dt>
<dd>
<p>The number of RPC handlers actively servicing requests</p>
</dd>
<dt class="hdlist1">hbase.regionserver.numCallsInGeneralQueue</dt>
<dd>
<p>The number of currently enqueued user requests</p>
</dd>
<dt class="hdlist1">hbase.regionserver.numCallsInReplicationQueue</dt>
<dd>
<p>The number of currently enqueued operations received from replication</p>
</dd>
<dt class="hdlist1">hbase.regionserver.numCallsInPriorityQueue</dt>
<dd>
<p>The number of currently enqueued priority (internal housekeeping) requests</p>
</dd>
<dt class="hdlist1">hbase.regionserver.flushQueueLength</dt>
<dd>
<p>Current depth of the memstore flush queue.
If increasing, we are falling behind with clearing memstores out to HDFS.</p>
</dd>
<dt class="hdlist1">hbase.regionserver.updatesBlockedTime</dt>
<dd>
<p>Number of milliseconds updates have been blocked so the memstore can be flushed</p>
</dd>
<dt class="hdlist1">hbase.regionserver.compactionQueueLength</dt>
<dd>
<p>Current depth of the compaction request queue.
If increasing, we are falling behind with storefile compaction.</p>
</dd>
<dt class="hdlist1">hbase.regionserver.blockCacheHitCount</dt>
<dd>
<p>The number of block cache hits</p>
</dd>
<dt class="hdlist1">hbase.regionserver.blockCacheMissCount</dt>
<dd>
<p>The number of block cache misses</p>
</dd>
<dt class="hdlist1">hbase.regionserver.blockCacheExpressHitPercent </dt>
<dd>
<p>The percent of the time that requests with the cache turned on hit the cache</p>
</dd>
<dt class="hdlist1">hbase.regionserver.percentFilesLocal</dt>
<dd>
<p>Percent of store file data that can be read from the local DataNode, 0-100</p>
</dd>
<dt class="hdlist1">hbase.regionserver.&lt;op&gt;_&lt;measure&gt;</dt>
<dd>
<p>Operation latencies, where &lt;op&gt; is one of Append, Delete, Mutate, Get, Replay, Increment; and where &lt;measure&gt; is one of min, max, mean, median, 75th_percentile, 95th_percentile, 99th_percentile</p>
</dd>
<dt class="hdlist1">hbase.regionserver.slow&lt;op&gt;Count </dt>
<dd>
<p>The number of operations we thought were slow, where &lt;op&gt; is one of the list above</p>
</dd>
<dt class="hdlist1">hbase.regionserver.GcTimeMillis</dt>
<dd>
<p>Time spent in garbage collection, in milliseconds</p>
</dd>
<dt class="hdlist1">hbase.regionserver.GcTimeMillisParNew</dt>
<dd>
<p>Time spent in garbage collection of the young generation, in milliseconds</p>
</dd>
<dt class="hdlist1">hbase.regionserver.GcTimeMillisConcurrentMarkSweep</dt>
<dd>
<p>Time spent in garbage collection of the old generation, in milliseconds</p>
</dd>
<dt class="hdlist1">hbase.regionserver.authenticationSuccesses</dt>
<dd>
<p>Number of client connections where authentication succeeded</p>
</dd>
<dt class="hdlist1">hbase.regionserver.authenticationFailures</dt>
<dd>
<p>Number of client connection authentication failures</p>
</dd>
<dt class="hdlist1">hbase.regionserver.mutationsWithoutWALCount </dt>
<dd>
<p>Count of writes submitted with a flag indicating they should bypass the write ahead log</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="rs_meta_metrics"><a class="anchor" href="#rs_meta_metrics"></a>162.7. Meta Table Load Metrics</h3>
<div class="paragraph">
<p>HBase meta table metrics collection feature is available in HBase 1.4+ but it is disabled by default, as it can
affect the performance of the cluster. When it is enabled, it helps to monitor client access patterns by collecting
the following statistics:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>number of get, put and delete operations on the <code>hbase:meta</code> table</p>
</li>
<li>
<p>number of get, put and delete operations made by the top-N clients</p>
</li>
<li>
<p>number of operations related to each table</p>
</li>
<li>
<p>number of operations related to the top-N regions</p>
<div class="dlist">
<dl>
<dt class="hdlist1">When to use the feature</dt>
<dd>
<p>This feature can help to identify hot spots in the meta table by showing the regions or tables where the meta info is
modified (e.g. by create, drop, split or move tables) or retrieved most frequently. It can also help to find misbehaving
client applications by showing which clients are using the meta table most heavily, which can for example suggest the
lack of meta table buffering or the lack of re-using open client connections in the client application.</p>
</dd>
</dl>
</div>
</li>
</ul>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Possible side-effects of enabling this feature</div>
<div class="paragraph">
<p>Having large number of clients and regions in the cluster can cause the registration and tracking of a large amount of
metrics, which can increase the memory and CPU footprint of the HBase region server handling the <code>hbase:meta</code> table.
It can also cause the significant increase of the JMX dump size, which can affect the monitoring or log aggregation
system you use beside HBase. It is recommended to turn on this feature only during debugging.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Where to find the metrics in JMX</dt>
<dd>
<p>Each metric attribute name will start with the ‘MetaTable_’ prefix. For all the metrics you will see five different
JMX attributes: count, mean rate, 1 minute rate, 5 minute rate and 15 minute rate. You will find these metrics in JMX
under the following MBean:
<code>Hadoop &#8594; HBase &#8594; RegionServer &#8594; Coprocessor.Region.CP_org.apache.hadoop.hbase.coprocessor.MetaTableMetrics</code>.</p>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="title">Examples: some Meta Table metrics you can see in your JMX dump</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="json">{
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_get_request_count</span><span class="delimiter">&quot;</span></span>: <span class="integer">77309</span>,
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_put_request_mean_rate</span><span class="delimiter">&quot;</span></span>: <span class="float">0.06339092997186495</span>,
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_table_MyTestTable_request_15min_rate</span><span class="delimiter">&quot;</span></span>: <span class="float">1.1020599841623246</span>,
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_client_/172.30.65.42_lossy_request_count</span><span class="delimiter">&quot;</span></span>: <span class="integer">1786</span>
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_client_/172.30.65.45_put_request_5min_rate</span><span class="delimiter">&quot;</span></span>: <span class="float">0.6189810954855728</span>,
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_region_1561131112259.c66e4308d492936179352c80432ccfe0._lossy_request_count</span><span class="delimiter">&quot;</span></span>: <span class="integer">38342</span>,
<span class="key"><span class="delimiter">&quot;</span><span class="content">MetaTable_region_1561131043640.5bdffe4b9e7e334172065c853cf0caa6._lossy_request_1min_rate</span><span class="delimiter">&quot;</span></span>: <span class="float">0.04925099917433935</span>,
}</code></pre>
</div>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Configuration</dt>
<dd>
<p>To turn on this feature, you have to enable a custom coprocessor by adding the following section to hbase-site.xml.
This coprocessor will run on all the HBase RegionServers, but will be active (i.e. consume memory / CPU) only on
the server, where the <code>hbase:meta</code> table is located. It will produce JMX metrics which can be downloaded from the
web UI of the given RegionServer or by a simple REST call. These metrics will not be present in the JMX dump of the
other RegionServers.</p>
</dd>
</dl>
</div>
<div class="listingblock">
<div class="title">Enabling the Meta Table Metrics feature</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.region.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.coprocessor.MetaTableMetrics<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">How the top-N metrics are calculated?</div>
<div class="paragraph">
<p>The 'top-N' type of metrics will be counted using the Lossy Counting Algorithm (as defined in
<a href="http://www.vldb.org/conf/2002/S10P03.pdf">Motwani, R; Manku, G.S (2002). "Approximate frequency counts over data streams"</a>),
which is designed to identify elements in a data stream whose frequency count exceed a user-given threshold.
The frequency computed by this algorithm is not always accurate but has an error threshold that can be specified by the
user as a configuration parameter. The run time space required by the algorithm is inversely proportional to the
specified error threshold, hence larger the error parameter, the smaller the footprint and the less accurate are the
metrics.</p>
</div>
<div class="paragraph">
<p>You can specify the error rate of the algorithm as a floating-point value between 0 and 1 (exclusive), it&#8217;s default
value is 0.02. Having the error rate set to <code>E</code> and having <code>N</code> as the total number of meta table operations, then
(assuming the uniform distribution of the activity of low frequency elements) at most <code>7 / E</code> meters will be kept and
each kept element will have a frequency higher than <code>E * N</code>.</p>
</div>
<div class="paragraph">
<p>An example: Let’s assume we are interested in the HBase clients that are most active in accessing the meta table.
When there was 1,000,000 operations on the meta table so far and the error rate parameter is set to 0.02, then we can
assume that only at most 350 client IP address related counters will be present in JMX and each of these clients
accessed the meta table at least 20,000 times.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.util.default.lossycounting.errorrate<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>0.02<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ops.monitoring"><a class="anchor" href="#ops.monitoring"></a>163. HBase Monitoring</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="ops.monitoring.overview"><a class="anchor" href="#ops.monitoring.overview"></a>163.1. Overview</h3>
<div class="paragraph">
<p>The following metrics are arguably the most important to monitor for each RegionServer for "macro monitoring", preferably with a system like <a href="http://opentsdb.net/">OpenTSDB</a>.
If your cluster is having performance issues it&#8217;s likely that you&#8217;ll see something unusual with this group.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">HBase</dt>
<dd>
<div class="ulist">
<ul>
<li>
<p>See <a href="#rs_metrics">rs metrics</a></p>
</li>
</ul>
</div>
</dd>
<dt class="hdlist1">OS</dt>
<dd>
<div class="ulist">
<ul>
<li>
<p>IO Wait</p>
</li>
<li>
<p>User CPU</p>
</li>
</ul>
</div>
</dd>
<dt class="hdlist1">Java</dt>
<dd>
<div class="ulist">
<ul>
<li>
<p>GC</p>
</li>
</ul>
</div>
</dd>
</dl>
</div>
<div class="paragraph">
<p>For more information on HBase metrics, see <a href="#hbase_metrics">hbase metrics</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.slow.query"><a class="anchor" href="#ops.slow.query"></a>163.2. Slow Query Log</h3>
<div class="paragraph">
<p>The HBase slow query log consists of parseable JSON structures describing the properties of those client operations (Gets, Puts, Deletes, etc.) that either took too long to run, or produced too much output.
The thresholds for "too long to run" and "too much output" are configurable, as described below.
The output is produced inline in the main region server logs so that it is easy to discover further details from context with other logged events.
It is also prepended with identifying tags <code>(responseTooSlow)</code>, <code>(responseTooLarge)</code>, <code>(operationTooSlow)</code>, and <code>(operationTooLarge)</code> in order to enable easy filtering with grep, in case the user desires to see only slow queries.</p>
</div>
<div class="sect3">
<h4 id="_configuration"><a class="anchor" href="#_configuration"></a>163.2.1. Configuration</h4>
<div class="paragraph">
<p>There are two configuration knobs that can be used to adjust the thresholds for when queries are logged.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hbase.ipc.warn.response.time</code> Maximum number of milliseconds that a query can be run without being logged.
Defaults to 10000, or 10 seconds.
Can be set to -1 to disable logging by time.</p>
</li>
<li>
<p><code>hbase.ipc.warn.response.size</code> Maximum byte size of response that a query can return without being logged.
Defaults to 100 megabytes.
Can be set to -1 to disable logging by size.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_metrics"><a class="anchor" href="#_metrics"></a>163.2.2. Metrics</h4>
<div class="paragraph">
<p>The slow query log exposes to metrics to JMX.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>hadoop.regionserver_rpc_slowResponse</code> a global metric reflecting the durations of all responses that triggered logging.</p>
</li>
<li>
<p><code>hadoop.regionserver_rpc_methodName.aboveOneSec</code> A metric reflecting the durations of all responses that lasted for more than one second.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_output"><a class="anchor" href="#_output"></a>163.2.3. Output</h4>
<div class="paragraph">
<p>The output is tagged with operation e.g. <code>(operationTooSlow)</code> if the call was a client operation, such as a Put, Get, or Delete, which we expose detailed fingerprint information for.
If not, it is tagged <code>(responseTooSlow)</code> and still produces parseable JSON output, but with less verbose information solely regarding its duration and size in the RPC itself. <code>TooLarge</code> is substituted for <code>TooSlow</code> if the response size triggered the logging, with <code>TooLarge</code> appearing even in the case that both size and duration triggered logging.</p>
</div>
</div>
<div class="sect3">
<h4 id="_example"><a class="anchor" href="#_example"></a>163.2.4. Example</h4>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2011</span>-<span class="integer">09</span>-<span class="integer">08</span> <span class="integer">10</span>:<span class="octal">01</span>:<span class="integer">25</span>,<span class="integer">824</span> WARN org.apache.hadoop.ipc.HBaseServer: (operationTooSlow): {<span class="string"><span class="delimiter">&quot;</span><span class="content">tables</span><span class="delimiter">&quot;</span></span>:{<span class="string"><span class="delimiter">&quot;</span><span class="content">riley2</span><span class="delimiter">&quot;</span></span>:{<span class="string"><span class="delimiter">&quot;</span><span class="content">puts</span><span class="delimiter">&quot;</span></span>:[{<span class="string"><span class="delimiter">&quot;</span><span class="content">totalColumns</span><span class="delimiter">&quot;</span></span>:<span class="integer">11</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">families</span><span class="delimiter">&quot;</span></span>:{<span class="string"><span class="delimiter">&quot;</span><span class="content">actions</span><span class="delimiter">&quot;</span></span>:[{<span class="string"><span class="delimiter">&quot;</span><span class="content">timestamp</span><span class="delimiter">&quot;</span></span>:<span class="integer">1315501284459</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">qualifier</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">0</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">vlen</span><span class="delimiter">&quot;</span></span>:<span class="integer">9667580</span>},{<span class="string"><span class="delimiter">&quot;</span><span class="content">timestamp</span><span class="delimiter">&quot;</span></span>:<span class="integer">1315501284459</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">qualifier</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">1</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">vlen</span><span class="delimiter">&quot;</span></span>:<span class="integer">10122412</span>},{<span class="string"><span class="delimiter">&quot;</span><span class="content">timestamp</span><span class="delimiter">&quot;</span></span>:<span class="integer">1315501284459</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">qualifier</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">2</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">vlen</span><span class="delimiter">&quot;</span></span>:<span class="integer">11104617</span>},{<span class="string"><span class="delimiter">&quot;</span><span class="content">timestamp</span><span class="delimiter">&quot;</span></span>:<span class="integer">1315501284459</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">qualifier</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">3</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">vlen</span><span class="delimiter">&quot;</span></span>:<span class="integer">13430635</span>}]},<span class="string"><span class="delimiter">&quot;</span><span class="content">row</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">cfcd208495d565ef66e7dff9f98764da:0</span><span class="delimiter">&quot;</span></span>}],<span class="string"><span class="delimiter">&quot;</span><span class="content">families</span><span class="delimiter">&quot;</span></span>:[<span class="string"><span class="delimiter">&quot;</span><span class="content">actions</span><span class="delimiter">&quot;</span></span>]}},<span class="string"><span class="delimiter">&quot;</span><span class="content">processingtimems</span><span class="delimiter">&quot;</span></span>:<span class="integer">956</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">client</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">10.47.34.63:33623</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">starttimems</span><span class="delimiter">&quot;</span></span>:<span class="integer">1315501284456</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">queuetimems</span><span class="delimiter">&quot;</span></span>:<span class="integer">0</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">totalPuts</span><span class="delimiter">&quot;</span></span>:<span class="integer">1</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">class</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">HRegionServer</span><span class="delimiter">&quot;</span></span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">responsesize</span><span class="delimiter">&quot;</span></span>:<span class="integer">0</span>,<span class="string"><span class="delimiter">&quot;</span><span class="content">method</span><span class="delimiter">&quot;</span></span>:<span class="string"><span class="delimiter">&quot;</span><span class="content">multiPut</span><span class="delimiter">&quot;</span></span>}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note that everything inside the "tables" structure is output produced by MultiPut&#8217;s fingerprint, while the rest of the information is RPC-specific, such as processing time and client IP/port.
Other client operations follow the same pattern and the same general structure, with necessary differences due to the nature of the individual operations.
In the case that the call is not a client operation, that detailed fingerprint information will be completely absent.</p>
</div>
<div class="paragraph">
<p>This particular example, for example, would indicate that the likely cause of slowness is simply a very large (on the order of 100MB) multiput, as we can tell by the "vlen," or value length, fields of each put in the multiPut.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_block_cache_monitoring"><a class="anchor" href="#_block_cache_monitoring"></a>163.3. Block Cache Monitoring</h3>
<div class="paragraph">
<p>Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
To view the block cache reports, see the Block Cache section of the region server UI.
Following are a few examples of the reporting capabilities.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/bc_basic.png" alt="bc basic">
</div>
<div class="title">Figure 9. Basic Info shows the cache implementation.</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/bc_config.png" alt="bc config">
</div>
<div class="title">Figure 10. Config shows all cache configuration options.</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/bc_stats.png" alt="bc stats">
</div>
<div class="title">Figure 11. Stats shows statistics about the performance of the cache.</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/bc_l1.png" alt="bc l1">
</div>
<div class="title">Figure 12. L1 and L2 show information about the L1 and L2 caches.</div>
</div>
<div class="paragraph">
<p>This is not an exhaustive list of all the screens and reports available.
Have a look in the Web UI.</p>
</div>
</div>
<div class="sect2">
<h3 id="_snapshot_space_usage_monitoring"><a class="anchor" href="#_snapshot_space_usage_monitoring"></a>163.4. Snapshot Space Usage Monitoring</h3>
<div class="paragraph">
<p>Starting with HBase 0.95, Snapshot usage information on individual snapshots was shown in the HBase Master Web UI. This was further enhanced starting with HBase 1.3 to show the total Storefile size of the Snapshot Set. The following metrics are shown in the Master Web UI with HBase 1.3 and later.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Shared Storefile Size is the Storefile size shared between snapshots and active tables.</p>
</li>
<li>
<p>Mob Storefile Size is the Mob Storefile size shared between snapshots and active tables.</p>
</li>
<li>
<p>Archived Storefile Size is the Storefile size in Archive.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The format of Archived Storefile Size is NNN(MMM). NNN is the total Storefile size in Archive, MMM is the total Storefile size in Archive that is specific to the snapshot (not shared with other snapshots and tables).</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/master-snapshot.png" alt="master snapshot">
</div>
<div class="title">Figure 13. Master Snapshot Overview</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/1-snapshot.png" alt="1 snapshot">
</div>
<div class="title">Figure 14. Snapshot Storefile Stats Example 1</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/2-snapshots.png" alt="2 snapshots">
</div>
<div class="title">Figure 15. Snapshot Storefile Stats Example 2</div>
</div>
<div class="imageblock">
<div class="content">
<img src="images/empty-snapshots.png" alt="empty snapshots">
</div>
<div class="title">Figure 16. Empty Snapshot Storfile Stats Example</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_cluster_replication"><a class="anchor" href="#_cluster_replication"></a>164. Cluster Replication</h2>
<div class="sectionbody">
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
This information was previously available at
<a href="https://hbase.apache.org/0.94/replication.html">Cluster Replication</a>.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>HBase provides a cluster replication mechanism which allows you to keep one cluster&#8217;s state synchronized with that of another cluster, using the write-ahead log (WAL) of the source cluster to propagate the changes.
Some use cases for cluster replication include:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Backup and disaster recovery</p>
</li>
<li>
<p>Data aggregation</p>
</li>
<li>
<p>Geographic data distribution</p>
</li>
<li>
<p>Online data ingestion combined with offline data analytics</p>
</li>
</ul>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Replication is enabled at the granularity of the column family.
Before enabling replication for a column family, create the table and all column families to be replicated, on the destination cluster.
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Replication is asynchronous as we send WAL to another cluster in background, which means that when you want to do recovery through replication, you could loss some data. To address this problem, we have introduced a new feature called synchronous replication. As the mechanism is a bit different so we use a separated section to describe it. Please see
<a href="#syncreplication">Synchronous Replication</a>.
</td>
</tr>
</table>
</div>
<div class="sect2">
<h3 id="_replication_overview"><a class="anchor" href="#_replication_overview"></a>164.1. Replication Overview</h3>
<div class="paragraph">
<p>Cluster replication uses a source-push methodology.
An HBase cluster can be a source (also called master or active, meaning that it is the originator of new data), a destination (also called slave or passive, meaning that it receives data via replication), or can fulfill both roles at once.
Replication is asynchronous, and the goal of replication is eventual consistency.
When the source receives an edit to a column family with replication enabled, that edit is propagated to all destination clusters using the WAL for that for that column family on the RegionServer managing the relevant region.</p>
</div>
<div class="paragraph">
<p>When data is replicated from one cluster to another, the original source of the data is tracked via a cluster ID which is part of the metadata.
In HBase 0.96 and newer (<a href="https://issues.apache.org/jira/browse/HBASE-7709">HBASE-7709</a>), all clusters which have already consumed the data are also tracked.
This prevents replication loops.</p>
</div>
<div class="paragraph">
<p>The WALs for each region server must be kept in HDFS as long as they are needed to replicate data to any slave cluster.
Each region server reads from the oldest log it needs to replicate and keeps track of its progress processing WALs inside ZooKeeper to simplify failure recovery.
The position marker which indicates a slave cluster&#8217;s progress, as well as the queue of WALs to process, may be different for every slave cluster.</p>
</div>
<div class="paragraph">
<p>The clusters participating in replication can be of different sizes.
The master cluster relies on randomization to attempt to balance the stream of replication on the slave clusters.
It is expected that the slave cluster has storage capacity to hold the replicated data, as well as any data it is responsible for ingesting.
If a slave cluster does run out of room, or is inaccessible for other reasons, it throws an error and the master retains the WAL and retries the replication at intervals.</p>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
<div class="title">Consistency Across Replicated Clusters</div>
<div class="paragraph">
<p>How your application builds on top of the HBase API matters when replication is in play. HBase&#8217;s replication system provides at-least-once delivery of client edits for an enabled column family to each configured destination cluster. In the event of failure to reach a given destination, the replication system will retry sending edits in a way that might repeat a given message. HBase provides two ways of replication, one is the original replication and the other is serial replication. In the previous way of replication, there is not a guaranteed order of delivery for client edits. In the event of a RegionServer failing, recovery of the replication queue happens independent of recovery of the individual regions that server was previously handling. This means that it is possible for the not-yet-replicated edits to be serviced by a RegionServer that is currently slower to replicate than the one that handles edits from after the failure.</p>
</div>
<div class="paragraph">
<p>The combination of these two properties (at-least-once delivery and the lack of message ordering) means that some destination clusters may end up in a different state if your application makes use of operations that are not idempotent, e.g. Increments.</p>
</div>
<div class="paragraph">
<p>To solve the problem, HBase now supports serial replication, which sends edits to destination cluster as the order of requests from client. See <a href="#_serial_replication">Serial Replication</a>.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Terminology Changes</div>
<div class="paragraph">
<p>Previously, terms such as <em class="firstterm">master-master</em>, <em class="firstterm">master-slave</em>, and <em class="firstterm">cyclical</em> were used to describe replication relationships in HBase.
These terms added confusion, and have been abandoned in favor of discussions about cluster topologies appropriate for different scenarios.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="ulist">
<div class="title">Cluster Topologies</div>
<ul>
<li>
<p>A central source cluster might propagate changes out to multiple destination clusters, for failover or due to geographic distribution.</p>
</li>
<li>
<p>A source cluster might push changes to a destination cluster, which might also push its own changes back to the original cluster.</p>
</li>
<li>
<p>Many different low-latency clusters might push changes to one centralized cluster for backup or resource-intensive data analytics jobs.
The processed data might then be replicated back to the low-latency clusters.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Multiple levels of replication may be chained together to suit your organization&#8217;s needs.
The following diagram shows a hypothetical scenario.
Use the arrows to follow the data paths.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/hbase_replication_diagram.jpg" alt="hbase replication diagram">
</div>
<div class="title">Figure 17. Example of a Complex Cluster Replication Configuration</div>
</div>
<div class="paragraph">
<p>HBase replication borrows many concepts from the <em class="firstterm">statement-based replication</em> design used by MySQL.
Instead of SQL statements, entire WALEdits (consisting of multiple cell inserts coming from Put and Delete operations on the clients) are replicated in order to maintain atomicity.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbase.replication.management"><a class="anchor" href="#hbase.replication.management"></a>164.2. Managing and Configuring Cluster Replication</h3>
<div class="olist arabic">
<div class="title">Cluster Configuration Overview</div>
<ol class="arabic">
<li>
<p>Configure and start the source and destination clusters.
Create tables with the same names and column families on both the source and destination clusters, so that the destination cluster knows where to store data it will receive.</p>
</li>
<li>
<p>All hosts in the source and destination clusters should be reachable to each other.</p>
</li>
<li>
<p>If both clusters use the same ZooKeeper cluster, you must use a different <code>zookeeper.znode.parent</code>, because they cannot write in the same folder.</p>
</li>
<li>
<p>On the source cluster, in HBase Shell, add the destination cluster as a peer, using the <code>add_peer</code> command.</p>
</li>
<li>
<p>On the source cluster, in HBase Shell, enable the table replication, using the <code>enable_table_replication</code> command.</p>
</li>
<li>
<p>Check the logs to see if replication is taking place. If so, you will see messages like the following, coming from the ReplicationSource.</p>
</li>
</ol>
</div>
<div class="listingblock">
<div class="content">
<pre>LOG.info("Replicating "+clusterId + " -&gt; " + peerClusterId);</pre>
</div>
</div>
<div class="paragraph">
<div class="title">Serial Replication Configuration</div>
<p>See <a href="#_serial_replication">Serial Replication</a></p>
</div>
<div class="dlist">
<div class="title">Cluster Management Commands</div>
<dl>
<dt class="hdlist1">add_peer &lt;ID&gt; &lt;CLUSTER_KEY&gt;</dt>
<dd>
<p>Adds a replication relationship between two clusters.<br></p>
<div class="ulist">
<ul>
<li>
<p>ID&#8201;&#8212;&#8201;a unique string, which must not contain a hyphen.</p>
</li>
<li>
<p>CLUSTER_KEY: composed using the following template, with appropriate place-holders: <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>. This value can be found on the Master UI info page.</p>
</li>
<li>
<p>STATE(optional): ENABLED or DISABLED, default value is ENABLED</p>
</li>
</ul>
</div>
</dd>
<dt class="hdlist1">list_peers</dt>
<dd>
<p>list all replication relationships known by this cluster</p>
</dd>
<dt class="hdlist1">enable_peer &lt;ID&gt;</dt>
<dd>
<p>Enable a previously-disabled replication relationship</p>
</dd>
<dt class="hdlist1">disable_peer &lt;ID&gt;</dt>
<dd>
<p>Disable a replication relationship. HBase will no longer send edits to that
peer cluster, but it still keeps track of all the new WALs that it will need
to replicate if and when it is re-enabled. WALs are retained when enabling or disabling
replication as long as peers exist.</p>
</dd>
<dt class="hdlist1">remove_peer &lt;ID&gt;</dt>
<dd>
<p>Disable and remove a replication relationship. HBase will no longer send edits to that peer cluster or keep track of WALs.</p>
</dd>
<dt class="hdlist1">enable_table_replication &lt;TABLE_NAME&gt;</dt>
<dd>
<p>Enable the table replication switch for all its column families. If the table is not found in the destination cluster then it will create one with the same name and column families.</p>
</dd>
<dt class="hdlist1">disable_table_replication &lt;TABLE_NAME&gt;</dt>
<dd>
<p>Disable the table replication switch for all its column families.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_serial_replication"><a class="anchor" href="#_serial_replication"></a>164.3. Serial Replication</h3>
<div class="paragraph">
<p>Note: this feature is introduced in HBase 2.1</p>
</div>
<div class="paragraph">
<div class="title">Function of serial replication</div>
<p>Serial replication supports to push logs to the destination cluster in the same order as logs reach to the source cluster.</p>
</div>
<div class="paragraph">
<div class="title">Why need serial replication?</div>
<p>In replication of HBase, we push mutations to destination cluster by reading WAL in each region server. We have a queue for WAL files so we can read them in order of creation time. However, when region-move or RS failure occurs in source cluster, the hlog entries that are not pushed before region-move or RS-failure will be pushed by original RS(for region move) or another RS which takes over the remained hlog of dead RS(for RS failure), and the new entries for the same region(s) will be pushed by the RS which now serves the region(s), but they push the hlog entries of a same region concurrently without coordination.</p>
</div>
<div class="paragraph">
<p>This treatment can possibly lead to data inconsistency between source and destination clusters:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>there are put and then delete written to source cluster.</p>
</li>
<li>
<p>due to region-move / RS-failure, they are pushed by different replication-source threads to peer cluster.</p>
</li>
<li>
<p>if delete is pushed to peer cluster before put, and flush and major-compact occurs in peer cluster before put is pushed to peer cluster, the delete is collected and the put remains in peer cluster, but in source cluster the put is masked by the delete, hence data inconsistency between source and destination clusters.</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Serial replication configuration</div>
<p>Set the serial flag to true for a repliation peer. And the default serial flag is false.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Add a new replication peer which serial flag is true</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; add_peer <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="constant">CLUSTER_KEY</span> =&gt; <span class="string"><span class="delimiter">&quot;</span><span class="content">server1.cie.com:2181:/hbase</span><span class="delimiter">&quot;</span></span>, <span class="constant">SERIAL</span> =&gt; <span class="predefined-constant">true</span></code></pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Set a replication peer&#8217;s serial flag to false</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; set_peer_serial <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="predefined-constant">false</span></code></pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Set a replication peer&#8217;s serial flag to true</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="ruby">hbase&gt; set_peer_serial <span class="string"><span class="delimiter">'</span><span class="content">1</span><span class="delimiter">'</span></span>, <span class="predefined-constant">true</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>The serial replication feature had been done firstly in <a href="https://issues.apache.org/jira/browse/HBASE-9465">HBASE-9465</a> and then reverted and redone in <a href="https://issues.apache.org/jira/browse/HBASE-20046">HBASE-20046</a>. You can find more details in these issues.</p>
</div>
</div>
<div class="sect2">
<h3 id="_verifying_replicated_data"><a class="anchor" href="#_verifying_replicated_data"></a>164.4. Verifying Replicated Data</h3>
<div class="paragraph">
<p>The <code>VerifyReplication</code> MapReduce job, which is included in HBase, performs a systematic comparison of replicated data between two different clusters. Run the VerifyReplication job on the master cluster, supplying it with the peer ID and table name to use for validation. You can limit the verification further by specifying a time range or specific families. The job&#8217;s short name is <code>verifyrep</code>. To run the job, use a command like the following:</p>
</div>
<div class="paragraph">
<p>+</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` &quot;${HADOOP_HOME}/bin/hadoop&quot; jar &quot;${HBASE_HOME}/hbase-mapreduce-VERSION.jar&quot; verifyrep --starttime=&lt;timestamp&gt; --endtime=&lt;timestamp&gt; --families=&lt;myFam&gt; &lt;ID&gt; &lt;tableName&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>+
The <code>VerifyReplication</code> command prints out <code>GOODROWS</code> and <code>BADROWS</code> counters to indicate rows that did and did not replicate correctly.</p>
</div>
</div>
<div class="sect2">
<h3 id="_detailed_information_about_cluster_replication"><a class="anchor" href="#_detailed_information_about_cluster_replication"></a>164.5. Detailed Information About Cluster Replication</h3>
<div class="imageblock">
<div class="content">
<img src="images/replication_overview.png" alt="replication overview">
</div>
<div class="title">Figure 18. Replication Architecture Overview</div>
</div>
<div class="sect3">
<h4 id="_life_of_a_wal_edit"><a class="anchor" href="#_life_of_a_wal_edit"></a>164.5.1. Life of a WAL Edit</h4>
<div class="paragraph">
<p>A single WAL edit goes through several steps in order to be replicated to a slave cluster.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>An HBase client uses a Put or Delete operation to manipulate data in HBase.</p>
</li>
<li>
<p>The region server writes the request to the WAL in a way allows it to be replayed if it is not written successfully.</p>
</li>
<li>
<p>If the changed cell corresponds to a column family that is scoped for replication, the edit is added to the queue for replication.</p>
</li>
<li>
<p>In a separate thread, the edit is read from the log, as part of a batch process.
Only the KeyValues that are eligible for replication are kept.
Replicable KeyValues are part of a column family whose schema is scoped GLOBAL, are not part of a catalog such as <code>hbase:meta</code>, did not originate from the target slave cluster, and have not already been consumed by the target slave cluster.</p>
</li>
<li>
<p>The edit is tagged with the master&#8217;s UUID and added to a buffer.
When the buffer is filled, or the reader reaches the end of the file, the buffer is sent to a random region server on the slave cluster.</p>
</li>
<li>
<p>The region server reads the edits sequentially and separates them into buffers, one buffer per table.
After all edits are read, each buffer is flushed using <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html">Table</a>, HBase&#8217;s normal client.
The master&#8217;s UUID and the UUIDs of slaves which have already consumed the data are preserved in the edits they are applied, in order to prevent replication loops.</p>
</li>
<li>
<p>In the master, the offset for the WAL that is currently being replicated is registered in ZooKeeper.</p>
</li>
<li>
<p>The first three steps, where the edit is inserted, are identical.</p>
</li>
<li>
<p>Again in a separate thread, the region server reads, filters, and edits the log edits in the same way as above.
The slave region server does not answer the RPC call.</p>
</li>
<li>
<p>The master sleeps and tries again a configurable number of times.</p>
</li>
<li>
<p>If the slave region server is still not available, the master selects a new subset of region server to replicate to, and tries again to send the buffer of edits.</p>
</li>
<li>
<p>Meanwhile, the WALs are rolled and stored in a queue in ZooKeeper.
Logs that are <em class="firstterm">archived</em> by their region server, by moving them from the region server&#8217;s log directory to a central log directory, will update their paths in the in-memory queue of the replicating thread.</p>
</li>
<li>
<p>When the slave cluster is finally available, the buffer is applied in the same way as during normal processing.
The master region server will then replicate the backlog of logs that accumulated during the outage.</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Spreading Queue Failover Load</div>
<p>When replication is active, a subset of region servers in the source cluster is responsible for shipping edits to the sink.
This responsibility must be failed over like all other region server functions should a process or node crash.
The following configuration settings are recommended for maintaining an even distribution of replication activity over the remaining live servers in the source cluster:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Set <code>replication.source.maxretriesmultiplier</code> to <code>300</code>.</p>
</li>
<li>
<p>Set <code>replication.source.sleepforretries</code> to <code>1</code> (1 second). This value, combined with the value of <code>replication.source.maxretriesmultiplier</code>, causes the retry cycle to last about 5 minutes.</p>
</li>
<li>
<p>Set <code>replication.sleep.before.failover</code> to <code>30000</code> (30 seconds) in the source cluster site configuration.</p>
</li>
</ul>
</div>
<div id="cluster.replication.preserving.tags" class="paragraph">
<div class="title">Preserving Tags During Replication</div>
<p>By default, the codec used for replication between clusters strips tags, such as cell-level ACLs, from cells.
To prevent the tags from being stripped, you can use a different codec which does not strip them.
Configure <code>hbase.replication.rpc.codec</code> to use <code>org.apache.hadoop.hbase.codec.KeyValueCodecWithTags</code>, on both the source and sink RegionServers involved in the replication.
This option was introduced in <a href="https://issues.apache.org/jira/browse/HBASE-10322">HBASE-10322</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="_replication_internals"><a class="anchor" href="#_replication_internals"></a>164.5.2. Replication Internals</h4>
<div class="dlist">
<dl>
<dt class="hdlist1">Replication State in ZooKeeper</dt>
<dd>
<p>HBase replication maintains its state in ZooKeeper.
By default, the state is contained in the base node <em>/hbase/replication</em>.
This node contains two child nodes, the <code>Peers</code> znode and the <code>RS</code> znode.</p>
</dd>
<dt class="hdlist1">The <code>Peers</code> Znode</dt>
<dd>
<p>The <code>peers</code> znode is stored in <em>/hbase/replication/peers</em> by default.
It consists of a list of all peer replication clusters, along with the status of each of them.
The value of each peer is its cluster key, which is provided in the HBase Shell.
The cluster key contains a list of ZooKeeper nodes in the cluster&#8217;s quorum, the client port for the ZooKeeper quorum, and the base znode for HBase in HDFS on that cluster.</p>
</dd>
<dt class="hdlist1">The <code>RS</code> Znode</dt>
<dd>
<p>The <code>rs</code> znode contains a list of WAL logs which need to be replicated.
This list is divided into a set of queues organized by region server and the peer cluster the region server is shipping the logs to.
The rs znode has one child znode for each region server in the cluster.
The child znode name is the region server&#8217;s hostname, client port, and start code.
This list includes both live and dead region servers.</p>
</dd>
</dl>
</div>
</div>
<div class="sect3">
<h4 id="_choosing_region_servers_to_replicate_to"><a class="anchor" href="#_choosing_region_servers_to_replicate_to"></a>164.5.3. Choosing Region Servers to Replicate To</h4>
<div class="paragraph">
<p>When a master cluster region server initiates a replication source to a slave cluster, it first connects to the slave&#8217;s ZooKeeper ensemble using the provided cluster key . It then scans the <em>rs/</em> directory to discover all the available sinks (region servers that are accepting incoming streams of edits to replicate) and randomly chooses a subset of them using a configured ratio which has a default value of 10%. For example, if a slave cluster has 150 machines, 15 will be chosen as potential recipient for edits that this master cluster region server sends.
Because this selection is performed by each master region server, the probability that all slave region servers are used is very high, and this method works for clusters of any size.
For example, a master cluster of 10 machines replicating to a slave cluster of 5 machines with a ratio of 10% causes the master cluster region servers to choose one machine each at random.</p>
</div>
<div class="paragraph">
<p>A ZooKeeper watcher is placed on the <em>${zookeeper.znode.parent}/rs</em> node of the slave cluster by each of the master cluster&#8217;s region servers.
This watch is used to monitor changes in the composition of the slave cluster.
When nodes are removed from the slave cluster, or if nodes go down or come back up, the master cluster&#8217;s region servers will respond by selecting a new pool of slave region servers to replicate to.</p>
</div>
</div>
<div class="sect3">
<h4 id="_keeping_track_of_logs"><a class="anchor" href="#_keeping_track_of_logs"></a>164.5.4. Keeping Track of Logs</h4>
<div class="paragraph">
<p>Each master cluster region server has its own znode in the replication znodes hierarchy.
It contains one znode per peer cluster (if 5 slave clusters, 5 znodes are created), and each of these contain a queue of WALs to process.
Each of these queues will track the WALs created by that region server, but they can differ in size.
For example, if one slave cluster becomes unavailable for some time, the WALs should not be deleted, so they need to stay in the queue while the others are processed.
See <a href="#rs.failover.details">rs.failover.details</a> for an example.</p>
</div>
<div class="paragraph">
<p>When a source is instantiated, it contains the current WAL that the region server is writing to.
During log rolling, the new file is added to the queue of each slave cluster&#8217;s znode just before it is made available.
This ensures that all the sources are aware that a new log exists before the region server is able to append edits into it, but this operations is now more expensive.
The queue items are discarded when the replication thread cannot read more entries from a file (because it reached the end of the last block) and there are other files in the queue.
This means that if a source is up to date and replicates from the log that the region server writes to, reading up to the "end" of the current file will not delete the item in the queue.</p>
</div>
<div class="paragraph">
<p>A log can be archived if it is no longer used or if the number of logs exceeds <code>hbase.regionserver.maxlogs</code> because the insertion rate is faster than regions are flushed.
When a log is archived, the source threads are notified that the path for that log changed.
If a particular source has already finished with an archived log, it will just ignore the message.
If the log is in the queue, the path will be updated in memory.
If the log is currently being replicated, the change will be done atomically so that the reader doesn&#8217;t attempt to open the file when has already been moved.
Because moving a file is a NameNode operation , if the reader is currently reading the log, it won&#8217;t generate any exception.</p>
</div>
</div>
<div class="sect3">
<h4 id="_reading_filtering_and_sending_edits"><a class="anchor" href="#_reading_filtering_and_sending_edits"></a>164.5.5. Reading, Filtering and Sending Edits</h4>
<div class="paragraph">
<p>By default, a source attempts to read from a WAL and ship log entries to a sink as quickly as possible.
Speed is limited by the filtering of log entries Only KeyValues that are scoped GLOBAL and that do not belong to catalog tables will be retained.
Speed is also limited by total size of the list of edits to replicate per slave, which is limited to 64 MB by default.
With this configuration, a master cluster region server with three slaves would use at most 192 MB to store data to replicate.
This does not account for the data which was filtered but not garbage collected.</p>
</div>
<div class="paragraph">
<p>Once the maximum size of edits has been buffered or the reader reaches the end of the WAL, the source thread stops reading and chooses at random a sink to replicate to (from the list that was generated by keeping only a subset of slave region servers). It directly issues a RPC to the chosen region server and waits for the method to return.
If the RPC was successful, the source determines whether the current file has been emptied or it contains more data which needs to be read.
If the file has been emptied, the source deletes the znode in the queue.
Otherwise, it registers the new offset in the log&#8217;s znode.
If the RPC threw an exception, the source will retry 10 times before trying to find a different sink.</p>
</div>
</div>
<div class="sect3">
<h4 id="_cleaning_logs"><a class="anchor" href="#_cleaning_logs"></a>164.5.6. Cleaning Logs</h4>
<div class="paragraph">
<p>If replication is not enabled, the master&#8217;s log-cleaning thread deletes old logs using a configured TTL.
This TTL-based method does not work well with replication, because archived logs which have exceeded their TTL may still be in a queue.
The default behavior is augmented so that if a log is past its TTL, the cleaning thread looks up every queue until it finds the log, while caching queues it has found.
If the log is not found in any queues, the log will be deleted.
The next time the cleaning process needs to look for a log, it starts by using its cached list.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
WALs are saved when replication is enabled or disabled as long as peers exist.
</td>
</tr>
</table>
</div>
</div>
<div class="sect3">
<h4 id="rs.failover.details"><a class="anchor" href="#rs.failover.details"></a>164.5.7. Region Server Failover</h4>
<div class="paragraph">
<p>When no region servers are failing, keeping track of the logs in ZooKeeper adds no value.
Unfortunately, region servers do fail, and since ZooKeeper is highly available, it is useful for managing the transfer of the queues in the event of a failure.</p>
</div>
<div class="paragraph">
<p>Each of the master cluster region servers keeps a watcher on every other region server, in order to be notified when one dies (just as the master does). When a failure happens, they all race to create a znode called <code>lock</code> inside the dead region server&#8217;s znode that contains its queues.
The region server that creates it successfully then transfers all the queues to its own znode, one at a time since ZooKeeper does not support renaming queues.
After queues are all transferred, they are deleted from the old location.
The znodes that were recovered are renamed with the ID of the slave cluster appended with the name of the dead server.</p>
</div>
<div class="paragraph">
<p>Next, the master cluster region server creates one new source thread per copied queue, and each of the source threads follows the read/filter/ship pattern.
The main difference is that those queues will never receive new data, since they do not belong to their new region server.
When the reader hits the end of the last log, the queue&#8217;s znode is deleted and the master cluster region server closes that replication source.</p>
</div>
<div class="paragraph">
<p>Given a master cluster with 3 region servers replicating to a single slave with id <code>2</code>, the following hierarchy represents what the znodes layout could be at some point in time.
The region servers' znodes all contain a <code>peers</code> znode which contains a single queue.
The znode names in the queues represent the actual file names on HDFS in the form <code>address,port.timestamp</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1234 (Contains a position)
1.1.1.1,60020.1265
1.1.1.2,60020,123456790/
2/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312
1.1.1.3,60020, 123456630/
2/
1.1.1.3,60020.1280 (Contains a position)</pre>
</div>
</div>
<div class="paragraph">
<p>Assume that 1.1.1.2 loses its ZooKeeper session.
The survivors will race to create a lock, and, arbitrarily, 1.1.1.3 wins.
It will then start transferring all the queues to its local peers znode by appending the name of the dead server.
Right before 1.1.1.3 is able to clean up the old znodes, the layout will look like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1234 (Contains a position)
1.1.1.1,60020.1265
1.1.1.2,60020,123456790/
lock
2/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312
1.1.1.3,60020,123456630/
2/
1.1.1.3,60020.1280 (Contains a position)
2-1.1.1.2,60020,123456790/
1.1.1.2,60020.1214 (Contains a position)
1.1.1.2,60020.1248
1.1.1.2,60020.1312</pre>
</div>
</div>
<div class="paragraph">
<p>Some time later, but before 1.1.1.3 is able to finish replicating the last WAL from 1.1.1.2, it dies too.
Some new logs were also created in the normal queues.
The last region server will then try to lock 1.1.1.3&#8217;s znode and will begin transferring all the queues.
The new layout will be:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>/hbase/replication/rs/
1.1.1.1,60020,123456780/
2/
1.1.1.1,60020.1378 (Contains a position)
2-1.1.1.3,60020,123456630/
1.1.1.3,60020.1325 (Contains a position)
1.1.1.3,60020.1401
2-1.1.1.2,60020,123456790-1.1.1.3,60020,123456630/
1.1.1.2,60020.1312 (Contains a position)
1.1.1.3,60020,123456630/
lock
2/
1.1.1.3,60020.1325 (Contains a position)
1.1.1.3,60020.1401
2-1.1.1.2,60020,123456790/
1.1.1.2,60020.1312 (Contains a position)</pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_replication_metrics"><a class="anchor" href="#_replication_metrics"></a>164.6. Replication Metrics</h3>
<div class="paragraph">
<p>The following metrics are exposed at the global region server level and at the peer level:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>source.sizeOfLogQueue</code></dt>
<dd>
<p>number of WALs to process (excludes the one which is being processed) at the Replication source</p>
</dd>
<dt class="hdlist1"><code>source.shippedOps</code></dt>
<dd>
<p>number of mutations shipped</p>
</dd>
<dt class="hdlist1"><code>source.logEditsRead</code></dt>
<dd>
<p>number of mutations read from WALs at the replication source</p>
</dd>
<dt class="hdlist1"><code>source.ageOfLastShippedOp</code></dt>
<dd>
<p>age of last batch that was shipped by the replication source</p>
</dd>
<dt class="hdlist1"><code>source.completedLogs</code></dt>
<dd>
<p>The number of write-ahead-log files that have completed their acknowledged sending to the peer associated with this source. Increments to this metric are a part of normal operation of HBase replication.</p>
</dd>
<dt class="hdlist1"><code>source.completedRecoverQueues</code></dt>
<dd>
<p>The number of recovery queues this source has completed sending to the associated peer. Increments to this metric are a part of normal recovery of HBase replication in the face of failed Region Servers.</p>
</dd>
<dt class="hdlist1"><code>source.uncleanlyClosedLogs</code></dt>
<dd>
<p>The number of write-ahead-log files the replication system considered completed after reaching the end of readable entries in the face of an uncleanly closed file.</p>
</dd>
<dt class="hdlist1"><code>source.ignoredUncleanlyClosedLogContentsInBytes</code></dt>
<dd>
<p>When a write-ahead-log file is not closed cleanly, there will likely be some entry that has been partially serialized. This metric contains the number of bytes of such entries the HBase replication system believes were remaining at the end of files skipped in the face of an uncleanly closed file. Those bytes should either be in different file or represent a client write that was not acknowledged.</p>
</dd>
<dt class="hdlist1"><code>source.restartedLogReading</code></dt>
<dd>
<p>The number of times the HBase replication system detected that it failed to correctly parse a cleanly closed write-ahead-log file. In this circumstance, the system replays the entire log from the beginning, ensuring that no edits fail to be acknowledged by the associated peer. Increments to this metric indicate that the HBase replication system is having difficulty correctly handling failures in the underlying distributed storage system. No dataloss should occur, but you should check Region Server log files for details of the failures.</p>
</dd>
<dt class="hdlist1"><code>source.repeatedLogFileBytes</code></dt>
<dd>
<p>When the HBase replication system determines that it needs to replay a given write-ahead-log file, this metric is incremented by the number of bytes the replication system believes had already been acknowledged by the associated peer prior to starting over.</p>
</dd>
<dt class="hdlist1"><code>source.closedLogsWithUnknownFileLength</code></dt>
<dd>
<p>Incremented when the HBase replication system believes it is at the end of a write-ahead-log file but it can not determine the length of that file in the underlying distributed storage system. Could indicate dataloss since the replication system is unable to determine if the end of readable entries lines up with the expected end of the file. You should check Region Server log files for details of the failures.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_replication_configuration_options"><a class="anchor" href="#_replication_configuration_options"></a>164.7. Replication Configuration Options</h3>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Option</th>
<th class="tableblock halign-left valign-top">Description</th>
<th class="tableblock halign-left valign-top">Default</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">zookeeper.znode.parent</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The name of the base ZooKeeper znode used for HBase</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">/hbase</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">zookeeper.znode.replication</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The name of the base znode used for replication</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">replication</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">zookeeper.znode.replication.peers</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The name of the peer znode</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">peers</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">zookeeper.znode.replication.peers.state</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The name of peer-state znode</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">peer-state</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">zookeeper.znode.replication.rs</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The name of the rs znode</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">rs</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">replication.sleep.before.failover</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">How many milliseconds a worker should sleep before attempting to replicate
a dead region server&#8217;s WAL queues.</p></td>
<td class="tableblock halign-left valign-top"></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">replication.executor.workers</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The number of region servers a given region server should attempt to
failover simultaneously.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">1</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect2">
<h3 id="_monitoring_replication_status"><a class="anchor" href="#_monitoring_replication_status"></a>164.8. Monitoring Replication Status</h3>
<div class="paragraph">
<p>You can use the HBase Shell command <code>status 'replication'</code> to monitor the replication status on your cluster. The command has three variations:
* <code>status 'replication'</code>&#8201;&#8212;&#8201;prints the status of each source and its sinks, sorted by hostname.
* <code>status 'replication', 'source'</code>&#8201;&#8212;&#8201;prints the status for each replication source, sorted by hostname.
* <code>status 'replication', 'sink'</code>&#8201;&#8212;&#8201;prints the status for each replication sink, sorted by hostname.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_running_multiple_workloads_on_a_single_cluster"><a class="anchor" href="#_running_multiple_workloads_on_a_single_cluster"></a>165. Running Multiple Workloads On a Single Cluster</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase provides the following mechanisms for managing the performance of a cluster
handling multiple workloads:
. <a href="#quota">Quotas</a>
. <a href="#request_queues">Request Queues</a>
. <a href="#multiple-typed-queues">Multiple-Typed Queues</a></p>
</div>
<div class="sect2">
<h3 id="quota"><a class="anchor" href="#quota"></a>165.1. Quotas</h3>
<div class="paragraph">
<p>HBASE-11598 introduces RPC quotas, which allow you to throttle requests based on
the following limits:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p><a href="#request-quotas">The number or size of requests(read, write, or read+write) in a given timeframe</a></p>
</li>
<li>
<p><a href="#namespace_quotas">The number of tables allowed in a namespace</a></p>
</li>
</ol>
</div>
<div class="paragraph">
<p>These limits can be enforced for a specified user, table, or namespace.</p>
</div>
<div class="paragraph">
<div class="title">Enabling Quotas</div>
<p>Quotas are disabled by default. To enable the feature, set the <code>hbase.quota.enabled</code>
property to <code>true</code> in <em>hbase-site.xml</em> file for all cluster nodes.</p>
</div>
<div class="olist arabic">
<div class="title">General Quota Syntax</div>
<ol class="arabic">
<li>
<p>THROTTLE_TYPE can be expressed as READ, WRITE, or the default type(read + write).</p>
</li>
<li>
<p>Timeframes can be expressed in the following units: <code>sec</code>, <code>min</code>, <code>hour</code>, <code>day</code></p>
</li>
<li>
<p>Request sizes can be expressed in the following units: <code>B</code> (bytes), <code>K</code> (kilobytes),
<code>M</code> (megabytes), <code>G</code> (gigabytes), <code>T</code> (terabytes), <code>P</code> (petabytes)</p>
</li>
<li>
<p>Numbers of requests are expressed as an integer followed by the string <code>req</code></p>
</li>
<li>
<p>Limits relating to time are expressed as req/time or size/time. For instance <code>10req/day</code>
or <code>100P/hour</code>.</p>
</li>
<li>
<p>Numbers of tables or regions are expressed as integers.</p>
</li>
</ol>
</div>
<div id="request-quotas" class="paragraph">
<div class="title">Setting Request Quotas</div>
<p>You can set quota rules ahead of time, or you can change the throttle at runtime. The change
will propagate after the quota refresh period has expired. This expiration period
defaults to 5 minutes. To change it, modify the <code>hbase.quota.refresh.period</code> property
in <code>hbase-site.xml</code>. This property is expressed in milliseconds and defaults to <code>300000</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre># Limit user u1 to 10 requests per second
hbase&gt; set_quota TYPE =&gt; THROTTLE, USER =&gt; 'u1', LIMIT =&gt; '10req/sec'
# Limit user u1 to 10 read requests per second
hbase&gt; set_quota TYPE =&gt; THROTTLE, THROTTLE_TYPE =&gt; READ, USER =&gt; 'u1', LIMIT =&gt; '10req/sec'
# Limit user u1 to 10 M per day everywhere
hbase&gt; set_quota TYPE =&gt; THROTTLE, USER =&gt; 'u1', LIMIT =&gt; '10M/day'
# Limit user u1 to 10 M write size per sec
hbase&gt; set_quota TYPE =&gt; THROTTLE, THROTTLE_TYPE =&gt; WRITE, USER =&gt; 'u1', LIMIT =&gt; '10M/sec'
# Limit user u1 to 5k per minute on table t2
hbase&gt; set_quota TYPE =&gt; THROTTLE, USER =&gt; 'u1', TABLE =&gt; 't2', LIMIT =&gt; '5K/min'
# Limit user u1 to 10 read requests per sec on table t2
hbase&gt; set_quota TYPE =&gt; THROTTLE, THROTTLE_TYPE =&gt; READ, USER =&gt; 'u1', TABLE =&gt; 't2', LIMIT =&gt; '10req/sec'
# Remove an existing limit from user u1 on namespace ns2
hbase&gt; set_quota TYPE =&gt; THROTTLE, USER =&gt; 'u1', NAMESPACE =&gt; 'ns2', LIMIT =&gt; NONE
# Limit all users to 10 requests per hour on namespace ns1
hbase&gt; set_quota TYPE =&gt; THROTTLE, NAMESPACE =&gt; 'ns1', LIMIT =&gt; '10req/hour'
# Limit all users to 10 T per hour on table t1
hbase&gt; set_quota TYPE =&gt; THROTTLE, TABLE =&gt; 't1', LIMIT =&gt; '10T/hour'
# Remove all existing limits from user u1
hbase&gt; set_quota TYPE =&gt; THROTTLE, USER =&gt; 'u1', LIMIT =&gt; NONE
# List all quotas for user u1 in namespace ns2
hbase&gt; list_quotas USER =&gt; 'u1, NAMESPACE =&gt; 'ns2'
# List all quotas for namespace ns2
hbase&gt; list_quotas NAMESPACE =&gt; 'ns2'
# List all quotas for table t1
hbase&gt; list_quotas TABLE =&gt; 't1'
# list all quotas
hbase&gt; list_quotas</pre>
</div>
</div>
<div class="paragraph">
<p>You can also place a global limit and exclude a user or a table from the limit by applying the
<code>GLOBAL_BYPASS</code> property.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; set_quota NAMESPACE =&gt; 'ns1', LIMIT =&gt; '100req/min' # a per-namespace request limit
hbase&gt; set_quota USER =&gt; 'u1', GLOBAL_BYPASS =&gt; true # user u1 is not affected by the limit</pre>
</div>
</div>
<div id="namespace_quotas" class="paragraph">
<div class="title">Setting Namespace Quotas</div>
<p>You can specify the maximum number of tables or regions allowed in a given namespace, either
when you create the namespace or by altering an existing namespace, by setting the
<code>hbase.namespace.quota.maxtables property</code> on the namespace.</p>
</div>
<div class="listingblock">
<div class="title">Limiting Tables Per Namespace</div>
<div class="content">
<pre># Create a namespace with a max of 5 tables
hbase&gt; create_namespace 'ns1', {'hbase.namespace.quota.maxtables'=&gt;'5'}
# Alter an existing namespace to have a max of 8 tables
hbase&gt; alter_namespace 'ns2', {METHOD =&gt; 'set', 'hbase.namespace.quota.maxtables'=&gt;'8'}
# Show quota information for a namespace
hbase&gt; describe_namespace 'ns2'
# Alter an existing namespace to remove a quota
hbase&gt; alter_namespace 'ns2', {METHOD =&gt; 'unset', NAME=&gt;'hbase.namespace.quota.maxtables'}</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Limiting Regions Per Namespace</div>
<div class="content">
<pre># Create a namespace with a max of 10 regions
hbase&gt; create_namespace 'ns1', {'hbase.namespace.quota.maxregions'=&gt;'10'
# Show quota information for a namespace
hbase&gt; describe_namespace 'ns1'
# Alter an existing namespace to have a max of 20 tables
hbase&gt; alter_namespace 'ns2', {METHOD =&gt; 'set', 'hbase.namespace.quota.maxregions'=&gt;'20'}
# Alter an existing namespace to remove a quota
hbase&gt; alter_namespace 'ns2', {METHOD =&gt; 'unset', NAME=&gt; 'hbase.namespace.quota.maxregions'}</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="request_queues"><a class="anchor" href="#request_queues"></a>165.2. Request Queues</h3>
<div class="paragraph">
<p>If no throttling policy is configured, when the RegionServer receives multiple requests,
they are now placed into a queue waiting for a free execution slot (HBASE-6721).
The simplest queue is a FIFO queue, where each request waits for all previous requests in the queue
to finish before running. Fast or interactive queries can get stuck behind large requests.</p>
</div>
<div class="paragraph">
<p>If you are able to guess how long a request will take, you can reorder requests by
pushing the long requests to the end of the queue and allowing short requests to preempt
them. Eventually, you must still execute the large requests and prioritize the new
requests behind them. The short requests will be newer, so the result is not terrible,
but still suboptimal compared to a mechanism which allows large requests to be split
into multiple smaller ones.</p>
</div>
<div class="paragraph">
<p>HBASE-10993 introduces such a system for deprioritizing long-running scanners. There
are two types of queues, <code>fifo</code> and <code>deadline</code>. To configure the type of queue used,
configure the <code>hbase.ipc.server.callqueue.type</code> property in <code>hbase-site.xml</code>. There
is no way to estimate how long each request may take, so de-prioritization only affects
scans, and is based on the number of “next” calls a scan request has made. An assumption
is made that when you are doing a full table scan, your job is not likely to be interactive,
so if there are concurrent requests, you can delay long-running scans up to a limit tunable by
setting the <code>hbase.ipc.server.queue.max.call.delay</code> property. The slope of the delay is calculated
by a simple square root of <code>(numNextCall * weight)</code> where the weight is
configurable by setting the <code>hbase.ipc.server.scan.vtime.weight</code> property.</p>
</div>
</div>
<div class="sect2">
<h3 id="multiple-typed-queues"><a class="anchor" href="#multiple-typed-queues"></a>165.3. Multiple-Typed Queues</h3>
<div class="paragraph">
<p>You can also prioritize or deprioritize different kinds of requests by configuring
a specified number of dedicated handlers and queues. You can segregate the scan requests
in a single queue with a single handler, and all the other available queues can service
short <code>Get</code> requests.</p>
</div>
<div class="paragraph">
<p>You can adjust the IPC queues and handlers based on the type of workload, using static
tuning options. This approach is an interim first step that will eventually allow
you to change the settings at runtime, and to dynamically adjust values based on the load.</p>
</div>
<div class="paragraph">
<div class="title">Multiple Queues</div>
<p>To avoid contention and separate different kinds of requests, configure the
<code>hbase.ipc.server.callqueue.handler.factor</code> property, which allows you to increase the number of
queues and control how many handlers can share the same queue., allows admins to increase the number
of queues and decide how many handlers share the same queue.</p>
</div>
<div class="paragraph">
<p>Using more queues reduces contention when adding a task to a queue or selecting it
from a queue. You can even configure one queue per handler. The trade-off is that
if some queues contain long-running tasks, a handler may need to wait to execute from that queue
rather than stealing from another queue which has waiting tasks.</p>
</div>
<div class="paragraph">
<div class="title">Read and Write Queues</div>
<p>With multiple queues, you can now divide read and write requests, giving more priority
(more queues) to one or the other type. Use the <code>hbase.ipc.server.callqueue.read.ratio</code>
property to choose to serve more reads or more writes.</p>
</div>
<div class="paragraph">
<div class="title">Get and Scan Queues</div>
<p>Similar to the read/write split, you can split gets and scans by tuning the <code>hbase.ipc.server.callqueue.scan.ratio</code>
property to give more priority to gets or to scans. A scan ratio of <code>0.1</code> will give
more queue/handlers to the incoming gets, which means that more gets can be processed
at the same time and that fewer scans can be executed at the same time. A value of
<code>0.9</code> will give more queue/handlers to scans, so the number of scans executed will
increase and the number of gets will decrease.</p>
</div>
</div>
<div class="sect2">
<h3 id="space-quotas"><a class="anchor" href="#space-quotas"></a>165.4. Space Quotas</h3>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-16961">HBASE-16961</a> introduces a new type of
quotas for HBase to leverage: filesystem quotas. These "space" quotas limit the amount of space
on the filesystem that HBase namespaces and tables can consume. If a user, malicious or ignorant,
has the ability to write data into HBase, with enough time, that user can effectively crash HBase
(or worse HDFS) by consuming all available space. When there is no filesystem space available,
HBase crashes because it can no longer create/sync data to the write-ahead log.</p>
</div>
<div class="paragraph">
<p>This feature allows a for a limit to be set on the size of a table or namespace. When a space quota is set
on a namespace, the quota&#8217;s limit applies to the sum of usage of all tables in that namespace.
When a table with a quota exists in a namespace with a quota, the table quota takes priority
over the namespace quota. This allows for a scenario where a large limit can be placed on
a collection of tables, but a single table in that collection can have a fine-grained limit set.</p>
</div>
<div class="paragraph">
<p>The existing <code>set_quota</code> and <code>list_quota</code> HBase shell commands can be used to interact with
space quotas. Space quotas are quotas with a <code>TYPE</code> of <code>SPACE</code> and have <code>LIMIT</code> and <code>POLICY</code>
attributes. The <code>LIMIT</code> is a string that refers to the amount of space on the filesystem
that the quota subject (e.g. the table or namespace) may consume. For example, valid values
of <code>LIMIT</code> are <code>'10G'</code>, <code>'2T'</code>, or <code>'256M'</code>. The <code>POLICY</code> refers to the action that HBase will
take when the quota subject&#8217;s usage exceeds the <code>LIMIT</code>. The following are valid <code>POLICY</code> values.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>NO_INSERTS</code> - No new data may be written (e.g. <code>Put</code>, <code>Increment</code>, <code>Append</code>).</p>
</li>
<li>
<p><code>NO_WRITES</code> - Same as <code>NO_INSERTS</code> but <code>Deletes</code> are also disallowed.</p>
</li>
<li>
<p><code>NO_WRITES_COMPACTIONS</code> - Same as <code>NO_WRITES</code> but compactions are also disallowed.</p>
<div class="ulist">
<ul>
<li>
<p>This policy only prevents user-submitted compactions. System can still run compactions.</p>
</li>
</ul>
</div>
</li>
<li>
<p><code>DISABLE</code> - The table(s) are disabled, preventing all read/write access.</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="title">Setting simple space quotas</div>
<div class="content">
<pre># Sets a quota on the table 't1' with a limit of 1GB, disallowing Puts/Increments/Appends when the table exceeds 1GB
hbase&gt; set_quota TYPE =&gt; SPACE, TABLE =&gt; 't1', LIMIT =&gt; '1G', POLICY =&gt; NO_INSERTS
# Sets a quota on the namespace 'ns1' with a limit of 50TB, disallowing Puts/Increments/Appends/Deletes
hbase&gt; set_quota TYPE =&gt; SPACE, NAMESPACE =&gt; 'ns1', LIMIT =&gt; '50T', POLICY =&gt; NO_WRITES
# Sets a quota on the table 't3' with a limit of 2TB, disallowing any writes and compactions when the table exceeds 2TB.
hbase&gt; set_quota TYPE =&gt; SPACE, TABLE =&gt; 't3', LIMIT =&gt; '2T', POLICY =&gt; NO_WRITES_COMPACTIONS
# Sets a quota on the table 't2' with a limit of 50GB, disabling the table when it exceeds 50GB
hbase&gt; set_quota TYPE =&gt; SPACE, TABLE =&gt; 't2', LIMIT =&gt; '50G', POLICY =&gt; DISABLE</pre>
</div>
</div>
<div class="paragraph">
<p>Consider the following scenario to set up quotas on a namespace, overriding the quota on tables in that namespace</p>
</div>
<div class="listingblock">
<div class="title">Table and Namespace space quotas</div>
<div class="content">
<pre>hbase&gt; create_namespace 'ns1'
hbase&gt; create 'ns1:t1'
hbase&gt; create 'ns1:t2'
hbase&gt; create 'ns1:t3'
hbase&gt; set_quota TYPE =&gt; SPACE, NAMESPACE =&gt; 'ns1', LIMIT =&gt; '100T', POLICY =&gt; NO_INSERTS
hbase&gt; set_quota TYPE =&gt; SPACE, TABLE =&gt; 'ns1:t2', LIMIT =&gt; '200G', POLICY =&gt; NO_WRITES
hbase&gt; set_quota TYPE =&gt; SPACE, TABLE =&gt; 'ns1:t3', LIMIT =&gt; '20T', POLICY =&gt; NO_WRITES</pre>
</div>
</div>
<div class="paragraph">
<p>In the above scenario, the tables in the namespace <code>ns1</code> will not be allowed to consume more than
100TB of space on the filesystem among each other. The table 'ns1:t2' is only allowed to be 200GB in size, and will
disallow all writes when the usage exceeds this limit. The table 'ns1:t3' is allowed to grow to 20TB in size
and also will disallow all writes then the usage exceeds this limit. Because there is no table quota
on 'ns1:t1', this table can grow up to 100TB, but only if 'ns1:t2' and 'ns1:t3' have a usage of zero bytes.
Practically, it&#8217;s limit is 100TB less the current usage of 'ns1:t2' and 'ns1:t3'.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.space.quota.deletion"><a class="anchor" href="#ops.space.quota.deletion"></a>165.5. Disabling Automatic Space Quota Deletion</h3>
<div class="paragraph">
<p>By default, if a table or namespace is deleted that has a space quota, the quota itself is
also deleted. In some cases, it may be desirable for the space quota to not be automatically deleted.
In these cases, the user may configure the system to not delete any space quota automatically via hbase-site.xml.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> &lt;property&gt;
&lt;name&gt;hbase.quota.remove.on.table.delete&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">false</span>&lt;/value&gt;
&lt;/property&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>The value is set to <code>true</code> by default.</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_snapshots_with_space_quotas"><a class="anchor" href="#_hbase_snapshots_with_space_quotas"></a>165.6. HBase Snapshots with Space Quotas</h3>
<div class="paragraph">
<p>One common area of unintended-filesystem-use with HBase is via HBase snapshots. Because snapshots
exist outside of the management of HBase tables, it is not uncommon for administrators to suddenly
realize that hundreds of gigabytes or terabytes of space is being used by HBase snapshots which were
forgotten and never removed.</p>
</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-17748">HBASE-17748</a> is the umbrella JIRA issue which
expands on the original space quota functionality to also include HBase snapshots. While this is a confusing
subject, the implementation attempts to present this support in as reasonable and simple of a manner as
possible for administrators. This feature does not make any changes to administrator interaction with
space quotas, only in the internal computation of table/namespace usage. Table and namespace usage will
automatically incorporate the size taken by a snapshot per the rules defined below.</p>
</div>
<div class="paragraph">
<p>As a review, let&#8217;s cover a snapshot&#8217;s lifecycle: a snapshot is metadata which points to
a list of HFiles on the filesystem. This is why creating a snapshot is a very cheap operation; no HBase
table data is actually copied to perform a snapshot. Cloning a snapshot into a new table or restoring
a table is a cheap operation for the same reason; the new table references the files which already exist
on the filesystem without a copy. To include snapshots in space quotas, we need to define which table
"owns" a file when a snapshot references the file ("owns" refers to encompassing the filesystem usage
of that file).</p>
</div>
<div class="paragraph">
<p>Consider a snapshot which was made against a table. When the snapshot refers to a file and the table no
longer refers to that file, the "originating" table "owns" that file. When multiple snapshots refer to
the same file and no table refers to that file, the snapshot with the lowest-sorting name (lexicographically)
is chosen and the table which that snapshot was created from "owns" that file. HFiles are not "double-counted"
hen a table and one or more snapshots refer to that HFile.</p>
</div>
<div class="paragraph">
<p>When a table is "rematerialized" (via <code>clone_snapshot</code> or <code>restore_snapshot</code>), a similar problem of file
ownership arises. In this case, while the rematerialized table references a file which a snapshot also
references, the table does not "own" the file. The table from which the snapshot was created still "owns"
that file. When the rematerialized table is compacted or the snapshot is deleted, the rematerialized table
will uniquely refer to a new file and "own" the usage of that file. Similarly, when a table is duplicated via a snapshot
and <code>restore_snapshot</code>, the new table will not consume any quota size until the original table stops referring
to the files, either due to a compaction on the original table, a compaction on the new table, or the
original table being deleted.</p>
</div>
<div class="paragraph">
<p>One new HBase shell command was added to inspect the computed sizes of each snapshot in an HBase instance.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; list_snapshot_sizes
SNAPSHOT SIZE
t1.s1 1159108</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ops.backup"><a class="anchor" href="#ops.backup"></a>166. HBase Backup</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are two broad strategies for performing HBase backups: backing up with a full cluster shutdown, and backing up on a live cluster.
Each approach has pros and cons.</p>
</div>
<div class="paragraph">
<p>For additional information, see <a href="http://blog.sematext.com/2011/03/11/hbase-backup-options/">HBase Backup
Options</a> over on the Sematext Blog.</p>
</div>
<div class="sect2">
<h3 id="ops.backup.fullshutdown"><a class="anchor" href="#ops.backup.fullshutdown"></a>166.1. Full Shutdown Backup</h3>
<div class="paragraph">
<p>Some environments can tolerate a periodic full shutdown of their HBase cluster, for example if it is being used a back-end analytic capacity and not serving front-end web-pages.
The benefits are that the NameNode/Master are RegionServers are down, so there is no chance of missing any in-flight changes to either StoreFiles or metadata.
The obvious con is that the cluster is down.
The steps include:</p>
</div>
<div class="sect3">
<h4 id="ops.backup.fullshutdown.stop"><a class="anchor" href="#ops.backup.fullshutdown.stop"></a>166.1.1. Stop HBase</h4>
</div>
<div class="sect3">
<h4 id="ops.backup.fullshutdown.distcp"><a class="anchor" href="#ops.backup.fullshutdown.distcp"></a>166.1.2. Distcp</h4>
<div class="paragraph">
<p>Distcp could be used to either copy the contents of the HBase directory in HDFS to either the same cluster in another directory, or to a different cluster.</p>
</div>
<div class="paragraph">
<p>Note: Distcp works in this situation because the cluster is down and there are no in-flight edits to files.
Distcp-ing of files in the HBase directory is not generally recommended on a live cluster.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.backup.fullshutdown.restore"><a class="anchor" href="#ops.backup.fullshutdown.restore"></a>166.1.3. Restore (if needed)</h4>
<div class="paragraph">
<p>The backup of the hbase directory from HDFS is copied onto the 'real' hbase directory via distcp.
The act of copying these files creates new HDFS metadata, which is why a restore of the NameNode edits from the time of the HBase backup isn&#8217;t required for this kind of restore, because it&#8217;s a restore (via distcp) of a specific HDFS directory (i.e., the HBase part) not the entire HDFS file-system.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.backup.live.replication"><a class="anchor" href="#ops.backup.live.replication"></a>166.2. Live Cluster Backup - Replication</h3>
<div class="paragraph">
<p>This approach assumes that there is a second cluster.
See the HBase page on <a href="https://hbase.apache.org/book.html#_cluster_replication">replication</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.backup.live.copytable"><a class="anchor" href="#ops.backup.live.copytable"></a>166.3. Live Cluster Backup - CopyTable</h3>
<div class="paragraph">
<p>The <a href="#copy.table">copytable</a> utility could either be used to copy data from one table to another on the same cluster, or to copy data to another table on another cluster.</p>
</div>
<div class="paragraph">
<p>Since the cluster is up, there is a risk that edits could be missed in the copy process.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.backup.live.export"><a class="anchor" href="#ops.backup.live.export"></a>166.4. Live Cluster Backup - Export</h3>
<div class="paragraph">
<p>The <a href="#export">export</a> approach dumps the content of a table to HDFS on the same cluster.
To restore the data, the <a href="#import">import</a> utility would be used.</p>
</div>
<div class="paragraph">
<p>Since the cluster is up, there is a risk that edits could be missed in the export process.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ops.snapshots"><a class="anchor" href="#ops.snapshots"></a>167. HBase Snapshots</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase Snapshots allow you to take a copy of a table (both contents and metadata)with a very small performance impact. A Snapshot is an immutable
collection of table metadata and a list of HFiles that comprised the table at the time the Snapshot was taken. A "clone"
of a snapshot creates a new table from that snapshot, and a "restore" of a snapshot returns the contents of a table to
what it was when the snapshot was created. The "clone" and "restore" operations do not require any data to be copied,
as the underlying HFiles (the files which contain the data for an HBase table) are not modified with either action.
Simiarly, exporting a snapshot to another cluster has little impact on RegionServers of the local cluster.</p>
</div>
<div class="paragraph">
<p>Prior to version 0.94.6, the only way to backup or to clone a table is to use CopyTable/ExportTable, or to copy all the hfiles in HDFS after disabling the table.
The disadvantages of these methods are that you can degrade region server performance (Copy/Export Table) or you need to disable the table, that means no reads or writes; and this is usually unacceptable.</p>
</div>
<div class="sect2">
<h3 id="ops.snapshots.configuration"><a class="anchor" href="#ops.snapshots.configuration"></a>167.1. Configuration</h3>
<div class="paragraph">
<p>To turn on the snapshot support just set the <code>hbase.snapshot.enabled</code> property to true.
(Snapshots are enabled by default in 0.95+ and off by default in 0.94.6+)</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> &lt;property&gt;
&lt;name&gt;hbase.snapshot.enabled&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">true</span>&lt;/value&gt;
&lt;/property&gt;</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.takeasnapshot"><a class="anchor" href="#ops.snapshots.takeasnapshot"></a>167.2. Take a Snapshot</h3>
<div class="paragraph">
<p>You can take a snapshot of a table regardless of whether it is enabled or disabled.
The snapshot operation doesn&#8217;t involve any data copying.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase&gt; snapshot 'myTable', 'myTableSnapshot-122112'</pre>
</div>
</div>
<div class="paragraph">
<div class="title">Take a Snapshot Without Flushing</div>
<p>The default behavior is to perform a flush of data in memory before the snapshot is taken.
This means that data in memory is included in the snapshot.
In most cases, this is the desired behavior.
However, if your set-up can tolerate data in memory being excluded from the snapshot, you can use the <code>SKIP_FLUSH</code> option of the <code>snapshot</code> command to disable and flushing while taking the snapshot.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; snapshot 'mytable', 'snapshot123', {SKIP_FLUSH =&gt; true}</pre>
</div>
</div>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
There is no way to determine or predict whether a very concurrent insert or update will be included in a given snapshot, whether flushing is enabled or disabled.
A snapshot is only a representation of a table during a window of time.
The amount of time the snapshot operation will take to reach each Region Server may vary from a few seconds to a minute, depending on the resource load and speed of the hardware or network, among other factors.
There is also no way to know whether a given insert or update is in memory or has been flushed.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<div class="title">Take a Snapshot With TTL</div>
<p>Snapshots have a lifecycle that is independent from the table from which they are created.
Although data in a table may be stored with TTL the data files containing them become
frozen by the snapshot. Space consumed by expired cells will not be reclaimed by normal
table housekeeping like compaction. While this is expected it can be inconvenient at scale.
When many snapshots are under management and the data in various tables is expired by
TTL some notion of optional TTL (and optional default TTL) for snapshots could be useful.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase&gt; snapshot 'mytable', 'snapshot1234', {TTL =&gt; 86400}</pre>
</div>
</div>
<div class="paragraph">
<p>The above command creates snapshot <code>snapshot1234</code> with TTL of 86400 sec(24 hours)
and hence, the snapshot is supposed to be cleaned up after 24 hours</p>
</div>
<div class="ulist">
<div class="title">Default Snapshot TTL:</div>
<ul>
<li>
<p>FOREVER by default</p>
</li>
<li>
<p>User specified Default TTL with config <code>hbase.master.snapshot.ttl</code></p>
</li>
</ul>
</div>
<div class="paragraph">
<p>While creating a Snapshot, if TTL in seconds is not specified, by default the snapshot
would not be deleted automatically. i.e. it would be retained forever until it is
manually deleted. However, the user can update this default TTL behavior by
providing default TTL in sec for key: <code>hbase.master.snapshot.ttl</code>.
Value 0 for this config indicates TTL: FOREVER</p>
</div>
<div class="paragraph">
<p>At any point of time, if Snapshot cleanup is supposed to be stopped due to
some snapshot restore activity, it is advisable to disable Snapshot Cleaner with
config:</p>
</div>
<div class="paragraph">
<p><code>hbase.master.cleaner.snapshot.disable</code>: "true"</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.list"><a class="anchor" href="#ops.snapshots.list"></a>167.3. Listing Snapshots</h3>
<div class="paragraph">
<p>List all snapshots taken (by printing the names and relative information).</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase&gt; list_snapshots</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.delete"><a class="anchor" href="#ops.snapshots.delete"></a>167.4. Deleting Snapshots</h3>
<div class="paragraph">
<p>You can remove a snapshot, and the files retained for that snapshot will be removed if no longer needed.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase&gt; delete_snapshot 'myTableSnapshot-122112'</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.clone"><a class="anchor" href="#ops.snapshots.clone"></a>167.5. Clone a table from snapshot</h3>
<div class="paragraph">
<p>From a snapshot you can create a new table (clone operation) with the same data that you had when the snapshot was taken.
The clone operation, doesn&#8217;t involve data copies, and a change to the cloned table doesn&#8217;t impact the snapshot or the original table.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase&gt; clone_snapshot 'myTableSnapshot-122112', 'myNewTestTable'</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.restore"><a class="anchor" href="#ops.snapshots.restore"></a>167.6. Restore a snapshot</h3>
<div class="paragraph">
<p>The restore operation requires the table to be disabled, and the table will be restored to the state at the time when the snapshot was taken, changing both data and schema if required.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ ./bin/hbase shell
hbase&gt; disable 'myTable'
hbase&gt; restore_snapshot 'myTableSnapshot-122112'</pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Since Replication works at log level and snapshots at file-system level, after a restore, the replicas will be in a different state from the master.
If you want to use restore, you need to stop replication and redo the bootstrap.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>In case of partial data-loss due to misbehaving client, instead of a full restore that requires the table to be disabled, you can clone the table from the snapshot and use a Map-Reduce job to copy the data that you need, from the clone to the main one.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.acls"><a class="anchor" href="#ops.snapshots.acls"></a>167.7. Snapshots operations and ACLs</h3>
<div class="paragraph">
<p>If you are using security with the AccessController Coprocessor (See <a href="#hbase.accesscontrol.configuration">hbase.accesscontrol.configuration</a>), only a global administrator can take, clone, or restore a snapshot, and these actions do not capture the ACL rights.
This means that restoring a table preserves the ACL rights of the existing table, while cloning a table creates a new table that has no ACL rights until the administrator adds them.</p>
</div>
</div>
<div class="sect2">
<h3 id="ops.snapshots.export"><a class="anchor" href="#ops.snapshots.export"></a>167.8. Export to another cluster</h3>
<div class="paragraph">
<p>The ExportSnapshot tool copies all the data related to a snapshot (hfiles, logs, snapshot metadata) to another cluster.
The tool executes a Map-Reduce job, similar to distcp, to copy files between the two clusters, and since it works at file-system level the hbase cluster does not have to be online.</p>
</div>
<div class="paragraph">
<p>To copy a snapshot called MySnapshot to an HBase cluster srv2 (hdfs:///srv2:8082/hbase) using 16 mappers:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16</code></pre>
</div>
</div>
<div class="paragraph">
<div class="title">Limiting Bandwidth Consumption</div>
<p>You can limit the bandwidth consumption when exporting a snapshot, by specifying the <code>-bandwidth</code> parameter, which expects an integer representing megabytes per second.
The following example limits the above example to 200 MB/sec.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 -bandwidth 200</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="snapshots_s3"><a class="anchor" href="#snapshots_s3"></a>167.9. Storing Snapshots in an Amazon S3 Bucket</h3>
<div class="paragraph">
<p>You can store and retrieve snapshots from Amazon S3, using the following procedure.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
You can also store snapshots in Microsoft Azure Blob Storage. See <a href="#snapshots_azure">Storing Snapshots in Microsoft Azure Blob Storage</a>.
</td>
</tr>
</table>
</div>
<div class="ulist">
<div class="title">Prerequisites</div>
<ul>
<li>
<p>You must be using HBase 1.0 or higher and Hadoop 2.6.1 or higher, which is the first
configuration that uses the Amazon AWS SDK.</p>
</li>
<li>
<p>You must use the <code>s3a://</code> protocol to connect to Amazon S3. The older <code>s3n://</code>
and <code>s3://</code> protocols have various limitations and do not use the Amazon AWS SDK.</p>
</li>
<li>
<p>The <code>s3a://</code> URI must be configured and available on the server where you run
the commands to export and restore the snapshot.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>After you have fulfilled the prerequisites, take the snapshot like you normally would.
Afterward, you can export it using the <code>org.apache.hadoop.hbase.snapshot.ExportSnapshot</code>
command like the one below, substituting your own <code>s3a://</code> path in the <code>copy-from</code>
or <code>copy-to</code> directive and substituting or modifying other options as required:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
-snapshot MySnapshot \
-copy-from hdfs://srv2:8082/hbase \
-copy-to s3a://&lt;bucket&gt;/&lt;namespace&gt;/hbase \
-chuser MyUser \
-chgroup MyGroup \
-chmod 700 \
-mappers 16</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
-snapshot MySnapshot
-copy-from s3a://&lt;bucket&gt;/&lt;namespace&gt;/hbase \
-copy-to hdfs://srv2:8082/hbase \
-chuser MyUser \
-chgroup MyGroup \
-chmod 700 \
-mappers 16</pre>
</div>
</div>
<div class="paragraph">
<p>You can also use the <code>org.apache.hadoop.hbase.snapshot.SnapshotInfo</code> utility with the <code>s3a://</code> path by including the
<code>-remote-dir</code> option.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ hbase org.apache.hadoop.hbase.snapshot.SnapshotInfo \
-remote-dir s3a://&lt;bucket&gt;/&lt;namespace&gt;/hbase \
-list-snapshots</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="snapshots_azure"><a class="anchor" href="#snapshots_azure"></a>168. Storing Snapshots in Microsoft Azure Blob Storage</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can store snapshots in Microsoft Azure Blog Storage using the same techniques
as in <a href="#snapshots_s3">Storing Snapshots in an Amazon S3 Bucket</a>.</p>
</div>
<div class="ulist">
<div class="title">Prerequisites</div>
<ul>
<li>
<p>You must be using HBase 1.2 or higher with Hadoop 2.7.1 or
higher. No version of HBase supports Hadoop 2.7.0.</p>
</li>
<li>
<p>Your hosts must be configured to be aware of the Azure blob storage filesystem.
See <a href="https://hadoop.apache.org/docs/r2.7.1/hadoop-azure/index.html" class="bare">https://hadoop.apache.org/docs/r2.7.1/hadoop-azure/index.html</a>.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>After you meet the prerequisites, follow the instructions
in <a href="#snapshots_s3">Storing Snapshots in an Amazon S3 Bucket</a>, replacingthe protocol specifier with <code>wasb://</code> or <code>wasbs://</code>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ops.capacity"><a class="anchor" href="#ops.capacity"></a>169. Capacity Planning and Region Sizing</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are several considerations when planning the capacity for an HBase cluster and performing the initial configuration.
Start with a solid understanding of how HBase handles data internally.</p>
</div>
<div class="sect2">
<h3 id="ops.capacity.nodes"><a class="anchor" href="#ops.capacity.nodes"></a>169.1. Node count and hardware/VM configuration</h3>
<div class="sect3">
<h4 id="ops.capacity.nodes.datasize"><a class="anchor" href="#ops.capacity.nodes.datasize"></a>169.1.1. Physical data size</h4>
<div class="paragraph">
<p>Physical data size on disk is distinct from logical size of your data and is affected by the following:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Increased by HBase overhead</p>
</li>
<li>
<p>See <a href="#keyvalue">keyvalue</a> and <a href="#keysize">keysize</a>.
At least 24 bytes per key-value (cell), can be more.
Small keys/values means more relative overhead.</p>
</li>
<li>
<p>KeyValue instances are aggregated into blocks, which are indexed.
Indexes also have to be stored.
Blocksize is configurable on a per-ColumnFamily basis.
See <a href="#regions.arch">regions.arch</a>.</p>
</li>
<li>
<p>Decreased by <a href="#compression">compression</a> and data block encoding, depending on data.
See also <a href="http://search-hadoop.com/m/lL12B1PFVhp1">this thread</a>.
You might want to test what compression and encoding (if any) make sense for your data.</p>
</li>
<li>
<p>Increased by size of region server <a href="#wal">wal</a> (usually fixed and negligible - less than half of RS memory size, per RS).</p>
</li>
<li>
<p>Increased by HDFS replication - usually x3.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Aside from the disk space necessary to store the data, one RS may not be able to serve arbitrarily large amounts of data due to some practical limits on region count and size (see <a href="#ops.capacity.regions">ops.capacity.regions</a>).</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.nodes.throughput"><a class="anchor" href="#ops.capacity.nodes.throughput"></a>169.1.2. Read/Write throughput</h4>
<div class="paragraph">
<p>Number of nodes can also be driven by required throughput for reads and/or writes.
The throughput one can get per node depends a lot on data (esp.
key/value sizes) and request patterns, as well as node and system configuration.
Planning should be done for peak load if it is likely that the load would be the main driver of the increase of the node count.
PerformanceEvaluation and <a href="#ycsb">ycsb</a> tools can be used to test single node or a test cluster.</p>
</div>
<div class="paragraph">
<p>For write, usually 5-15Mb/s per RS can be expected, since every region server has only one active WAL.
There&#8217;s no good estimate for reads, as it depends vastly on data, requests, and cache hit rate. <a href="#perf.casestudy">perf.casestudy</a> might be helpful.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.nodes.gc"><a class="anchor" href="#ops.capacity.nodes.gc"></a>169.1.3. JVM GC limitations</h4>
<div class="paragraph">
<p>RS cannot currently utilize very large heap due to cost of GC.
There&#8217;s also no good way of running multiple RS-es per server (other than running several VMs per machine). Thus, ~20-24Gb or less memory dedicated to one RS is recommended.
GC tuning is required for large heap sizes.
See <a href="#gcpause">gcpause</a>, <a href="#trouble.log.gc">trouble.log.gc</a> and elsewhere (TODO: where?)</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.capacity.regions"><a class="anchor" href="#ops.capacity.regions"></a>169.2. Determining region count and size</h3>
<div class="paragraph">
<p>Generally less regions makes for a smoother running cluster (you can always manually split the big regions later (if necessary) to spread the data, or request load, over the cluster); 20-200 regions per RS is a reasonable range.
The number of regions cannot be configured directly (unless you go for fully <a href="#disable.splitting">disable.splitting</a>); adjust the region size to achieve the target region size given table size.</p>
</div>
<div class="paragraph">
<p>When configuring regions for multiple tables, note that most region settings can be set on a per-table basis via <a href="https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HTableDescriptor.html">HTableDescriptor</a>, as well as shell commands.
These settings will override the ones in <code>hbase-site.xml</code>.
That is useful if your tables have different workloads/use cases.</p>
</div>
<div class="paragraph">
<p>Also note that in the discussion of region sizes here, <em>HDFS replication factor is not (and should not be) taken into account, whereas
other factors <a href="#ops.capacity.nodes.datasize">ops.capacity.nodes.datasize</a> should be.</em> So, if your data is compressed and replicated 3 ways by HDFS, "9 Gb region" means 9 Gb of compressed data.
HDFS replication factor only affects your disk usage and is invisible to most HBase code.</p>
</div>
<div class="sect3">
<h4 id="_viewing_the_current_number_of_regions"><a class="anchor" href="#_viewing_the_current_number_of_regions"></a>169.2.1. Viewing the Current Number of Regions</h4>
<div class="paragraph">
<p>You can view the current number of regions for a given table using the HMaster UI.
In the <span class="label">Tables</span> section, the number of online regions for each table is listed in the <span class="label">Online Regions</span> column.
This total only includes the in-memory state and does not include disabled or offline regions.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.regions.count"><a class="anchor" href="#ops.capacity.regions.count"></a>169.2.2. Number of regions per RS - upper bound</h4>
<div class="paragraph">
<p>In production scenarios, where you have a lot of data, you are normally concerned with the maximum number of regions you can have per server. <a href="#too_many_regions">too many regions</a> has technical discussion on the subject.
Basically, the maximum number of regions is mostly determined by memstore memory usage.
Each region has its own memstores; these grow up to a configurable size; usually in 128-256 MB range, see <a href="#hbase.hregion.memstore.flush.size">hbase.hregion.memstore.flush.size</a>.
One memstore exists per column family (so there&#8217;s only one per region if there&#8217;s one CF in the table). The RS dedicates some fraction of total memory to its memstores (see <a href="#hbase.regionserver.global.memstore.size">hbase.regionserver.global.memstore.size</a>). If this memory is exceeded (too much memstore usage), it can cause undesirable consequences such as unresponsive server or compaction storms.
A good starting point for the number of regions per RS (assuming one table) is:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">((RS memory) * (total memstore fraction)) / ((memstore size)*(<span class="error">#</span> column families))</code></pre>
</div>
</div>
<div class="paragraph">
<p>This formula is pseudo-code.
Here are two formulas using the actual tunable parameters, first for HBase 0.98+ and second for HBase 0.94.x.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">HBase 0.98.x</dt>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre>((RS Xmx) * hbase.regionserver.global.memstore.size) / (hbase.hregion.memstore.flush.size * (# column families))</pre>
</div>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">HBase 0.94.x</dt>
</dl>
</div>
<div class="listingblock">
<div class="content">
<pre>((RS Xmx) * hbase.regionserver.global.memstore.upperLimit) / (hbase.hregion.memstore.flush.size * (# column families))+</pre>
</div>
</div>
<div class="paragraph">
<p>If a given RegionServer has 16 GB of RAM, with default settings, the formula works out to 16384*0.4/128 ~ 51 regions per RS is a starting point.
The formula can be extended to multiple tables; if they all have the same configuration, just use the total number of families.</p>
</div>
<div class="paragraph">
<p>This number can be adjusted; the formula above assumes all your regions are filled at approximately the same rate.
If only a fraction of your regions are going to be actively written to, you can divide the result by that fraction to get a larger region count.
Then, even if all regions are written to, all region memstores are not filled evenly, and eventually jitter appears even if they are (due to limited number of concurrent flushes). Thus, one can have as many as 2-3 times more regions than the starting point; however, increased numbers carry increased risk.</p>
</div>
<div class="paragraph">
<p>For write-heavy workload, memstore fraction can be increased in configuration at the expense of block cache; this will also allow one to have more regions.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.regions.mincount"><a class="anchor" href="#ops.capacity.regions.mincount"></a>169.2.3. Number of regions per RS - lower bound</h4>
<div class="paragraph">
<p>HBase scales by having regions across many servers.
Thus if you have 2 regions for 16GB data, on a 20 node machine your data will be concentrated on just a few machines - nearly the entire cluster will be idle.
This really can&#8217;t be stressed enough, since a common problem is loading 200MB data into HBase and then wondering why your awesome 10 node cluster isn&#8217;t doing anything.</p>
</div>
<div class="paragraph">
<p>On the other hand, if you have a very large amount of data, you may also want to go for a larger number of regions to avoid having regions that are too large.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.regions.size"><a class="anchor" href="#ops.capacity.regions.size"></a>169.2.4. Maximum region size</h4>
<div class="paragraph">
<p>For large tables in production scenarios, maximum region size is mostly limited by compactions - very large compactions, esp.
major, can degrade cluster performance.
Currently, the recommended maximum region size is 10-20Gb, and 5-10Gb is optimal.
For older 0.90.x codebase, the upper-bound of regionsize is about 4Gb, with a default of 256Mb.</p>
</div>
<div class="paragraph">
<p>The size at which the region is split into two is generally configured via <a href="#hbase.hregion.max.filesize">hbase.hregion.max.filesize</a>; for details, see <a href="#arch.region.splits">arch.region.splits</a>.</p>
</div>
<div class="paragraph">
<p>If you cannot estimate the size of your tables well, when starting off, it&#8217;s probably best to stick to the default region size, perhaps going smaller for hot tables (or manually split hot regions to spread the load over the cluster), or go with larger region sizes if your cell sizes tend to be largish (100k and up).</p>
</div>
<div class="paragraph">
<p>In HBase 0.98, experimental stripe compactions feature was added that would allow for larger regions, especially for log data.
See <a href="#ops.stripe">ops.stripe</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.regions.total"><a class="anchor" href="#ops.capacity.regions.total"></a>169.2.5. Total data size per region server</h4>
<div class="paragraph">
<p>According to above numbers for region size and number of regions per region server, in an optimistic estimate 10 GB x 100 regions per RS will give up to 1TB served per region server, which is in line with some of the reported multi-PB use cases.
However, it is important to think about the data vs cache size ratio at the RS level.
With 1TB of data per server and 10 GB block cache, only 1% of the data will be cached, which may barely cover all block indices.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="ops.capacity.config"><a class="anchor" href="#ops.capacity.config"></a>169.3. Initial configuration and tuning</h3>
<div class="paragraph">
<p>First, see <a href="#important_configurations">important configurations</a>.
Note that some configurations, more than others, depend on specific scenarios.
Pay special attention to:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><a href="#hbase.regionserver.handler.count">hbase.regionserver.handler.count</a> - request handler thread count, vital for high-throughput workloads.</p>
</li>
<li>
<p><a href="#config.wals">config.wals</a> - the blocking number of WAL files depends on your memstore configuration and should be set accordingly to prevent potential blocking when doing high volume of writes.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Then, there are some considerations when setting up your cluster and tables.</p>
</div>
<div class="sect3">
<h4 id="ops.capacity.config.compactions"><a class="anchor" href="#ops.capacity.config.compactions"></a>169.3.1. Compactions</h4>
<div class="paragraph">
<p>Depending on read/write volume and latency requirements, optimal compaction settings may be different.
See <a href="#compaction">compaction</a> for some details.</p>
</div>
<div class="paragraph">
<p>When provisioning for large data sizes, however, it&#8217;s good to keep in mind that compactions can affect write throughput.
Thus, for write-intensive workloads, you may opt for less frequent compactions and more store files per regions.
Minimum number of files for compactions (<code>hbase.hstore.compaction.min</code>) can be set to higher value; <a href="#hbase.hstore.blockingStoreFiles">hbase.hstore.blockingStoreFiles</a> should also be increased, as more files might accumulate in such case.
You may also consider manually managing compactions: <a href="#managed.compactions">managed.compactions</a></p>
</div>
</div>
<div class="sect3">
<h4 id="ops.capacity.config.presplit"><a class="anchor" href="#ops.capacity.config.presplit"></a>169.3.2. Pre-splitting the table</h4>
<div class="paragraph">
<p>Based on the target number of the regions per RS (see <a href="#ops.capacity.regions.count">ops.capacity.regions.count</a>) and number of RSes, one can pre-split the table at creation time.
This would both avoid some costly splitting as the table starts to fill up, and ensure that the table starts out already distributed across many servers.</p>
</div>
<div class="paragraph">
<p>If the table is expected to grow large enough to justify that, at least one region per RS should be created.
It is not recommended to split immediately into the full target number of regions (e.g.
50 * number of RSes), but a low intermediate value can be chosen.
For multiple tables, it is recommended to be conservative with presplitting (e.g.
pre-split 1 region per RS at most), especially if you don&#8217;t know how much each table will grow.
If you split too much, you may end up with too many regions, with some tables having too many small regions.</p>
</div>
<div class="paragraph">
<p>For pre-splitting howto, see <a href="#manual_region_splitting_decisions">manual region splitting decisions</a> and <a href="#precreate.regions">precreate.regions</a>.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="table.rename"><a class="anchor" href="#table.rename"></a>170. Table Rename</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In versions 0.90.x of hbase and earlier, we had a simple script that would rename the hdfs table directory and then do an edit of the hbase:meta table replacing all mentions of the old table name with the new.
The script was called <code>./bin/rename_table.rb</code>.
The script was deprecated and removed mostly because it was unmaintained and the operation performed by the script was brutal.</p>
</div>
<div class="paragraph">
<p>As of hbase 0.94.x, you can use the snapshot facility renaming a table.
Here is how you would do it using the hbase shell:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase shell&gt; disable 'tableName'
hbase shell&gt; snapshot 'tableName', 'tableSnapshot'
hbase shell&gt; clone_snapshot 'tableSnapshot', 'newTableName'
hbase shell&gt; delete_snapshot 'tableSnapshot'
hbase shell&gt; drop 'tableName'</pre>
</div>
</div>
<div class="paragraph">
<p>or in code it would be as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="type">void</span> rename(Admin admin, <span class="predefined-type">String</span> oldTableName, TableName newTableName) {
<span class="predefined-type">String</span> snapshotName = randomName();
admin.disableTable(oldTableName);
admin.snapshot(snapshotName, oldTableName);
admin.cloneSnapshot(snapshotName, newTableName);
admin.deleteSnapshot(snapshotName);
admin.deleteTable(oldTableName);
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="rsgroup"><a class="anchor" href="#rsgroup"></a>171. RegionServer Grouping</h2>
<div class="sectionbody">
<div class="paragraph">
<p>RegionServer Grouping (A.K.A <code>rsgroup</code>) is an advanced feature for
partitioning regionservers into distinctive groups for strict isolation. It
should only be used by users who are sophisticated enough to understand the
full implications and have a sufficient background in managing HBase clusters.
It was developed by Yahoo! and they run it at scale on their large grid cluster.
See <a href="http://www.slideshare.net/HBaseCon/keynote-apache-hbase-at-yahoo-scale">HBase at Yahoo! Scale</a>.</p>
</div>
<div class="paragraph">
<p>RSGroups are defined and managed with shell commands. The shell drives a
Coprocessor Endpoint whose API is marked private given this is an evolving
feature; the Coprocessor API is not for public consumption.
A server can be added to a group with hostname and port pair and tables
can be moved to this group so that only regionservers in the same rsgroup can
host the regions of the table. RegionServers and tables can only belong to one
rsgroup at a time. By default, all tables and regionservers belong to the
<code>default</code> rsgroup. System tables can also be put into a rsgroup using the regular
APIs. A custom balancer implementation tracks assignments per rsgroup and makes
sure to move regions to the relevant regionservers in that rsgroup. The rsgroup
information is stored in a regular HBase table, and a zookeeper-based read-only
cache is used at cluster bootstrap time.</p>
</div>
<div class="paragraph">
<p>To enable, add the following to your hbase-site.xml and restart your Master:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"> <span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.coprocessor.master.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.rsgroup.RSGroupAdminEndpoint<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.master.loadbalancer.class<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Then use the shell <em>rsgroup</em> commands to create and manipulate RegionServer
groups: e.g. to add a rsgroup and then add a server to it. To see the list of
rsgroup commands available in the hbase shell type:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash"> hbase(main):008:0&gt; help ‘rsgroup’
Took 0.5610 seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>High level, you create a rsgroup that is other than the <code>default</code> group using
<em>add_rsgroup</em> command. You then add servers and tables to this group with the
<em>move_servers_rsgroup</em> and <em>move_tables_rsgroup</em> commands. If necessary, run
a balance for the group if tables are slow to migrate to the groups dedicated
server with the <em>balance_rsgroup</em> command (Usually this is not needed). To
monitor effect of the commands, see the <code>Tables</code> tab toward the end of the
Master UI home page. If you click on a table, you can see what servers it is
deployed across. You should see here a reflection of the grouping done with
your shell commands. View the master log if issues.</p>
</div>
<div class="paragraph">
<p>Here is example using a few of the rsgroup commands. To add a group, do as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash"> hbase(main):008:0&gt; add_rsgroup 'my_group'
Took 0.5610 seconds</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">RegionServer Groups must be Enabled</div>
<div class="paragraph">
<p>If you have not enabled the rsgroup Coprocessor Endpoint in the master and
you run the any of the rsgroup shell commands, you will see an error message
like the below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">ERROR: org.apache.hadoop.hbase.exceptions.UnknownProtocolException: No registered master coprocessor service found <span class="keyword">for</span> name RSGroupAdminService
at org.apache.hadoop.hbase.master.MasterRpcServices.execMasterService(MasterRpcServices.java:<span class="integer">604</span>)
at org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos<span class="error">$</span>MasterService<span class="error">$</span><span class="integer">2</span>.callBlockingMethod(MasterProtos.java)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:<span class="integer">1140</span>)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:<span class="integer">133</span>)
at org.apache.hadoop.hbase.ipc.RpcExecutor<span class="error">$</span><span class="predefined-type">Handler</span>.run(RpcExecutor.java:<span class="integer">277</span>)
at org.apache.hadoop.hbase.ipc.RpcExecutor<span class="error">$</span><span class="predefined-type">Handler</span>.run(RpcExecutor.java:<span class="integer">257</span>)</code></pre>
</div>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Add a server (specified by hostname + port) to the just-made group using the
<em>move_servers_rsgroup</em> command as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash"> hbase(main):010:0&gt; move_servers_rsgroup 'my_group',['k.att.net:51129']</code></pre>
</div>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Hostname and Port vs ServerName</div>
<div class="paragraph">
<p>The rsgroup feature refers to servers in a cluster with hostname and port only.
It does not make use of the HBase ServerName type identifying RegionServers;
i.e. hostname + port + starttime to distinguish RegionServer instances. The
rsgroup feature keeps working across RegionServer restarts so the starttime of
ServerName&#8201;&#8212;&#8201;and hence the ServerName type&#8201;&#8212;&#8201;is not appropriate.
Administration</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Servers come and go over the lifetime of a Cluster. Currently, you must
manually align the servers referenced in rsgroups with the actual state of
nodes in the running cluster. What we mean by this is that if you decommission
a server, then you must update rsgroups as part of your server decommission
process removing references.</p>
</div>
<div class="paragraph">
<p>But, there is no _remove_offline_servers_rsgroup_command you say!</p>
</div>
<div class="paragraph">
<p>The way to remove a server is to move it to the <code>default</code> group. The <code>default</code>
group is special. All rsgroups, but the <code>default</code> rsgroup, are static in that
edits via the shell commands are persisted to the system <code>hbase:rsgroup</code> table.
If they reference a decommissioned server, then they need to be updated to undo
the reference.</p>
</div>
<div class="paragraph">
<p>The <code>default</code> group is not like other rsgroups in that it is dynamic. Its server
list mirrors the current state of the cluster; i.e. if you shutdown a server that
was part of the <code>default</code> rsgroup, and then do a <em>get_rsgroup</em> <code>default</code> to list
its content in the shell, the server will no longer be listed. For non-<code>default</code>
groups, though a mode may be offline, it will persist in the non-<code>default</code> group’s
list of servers. But if you move the offline server from the non-default rsgroup
to default, it will not show in the <code>default</code> list. It will just be dropped.</p>
</div>
<div class="sect2">
<h3 id="_best_practice"><a class="anchor" href="#_best_practice"></a>171.1. Best Practice</h3>
<div class="paragraph">
<p>The authors of the rsgroup feature, the Yahoo! HBase Engineering team, have been
running it on their grid for a good while now and have come up with a few best
practices informed by their experience.</p>
</div>
<div class="sect3">
<h4 id="_isolate_system_tables"><a class="anchor" href="#_isolate_system_tables"></a>171.1.1. Isolate System Tables</h4>
<div class="paragraph">
<p>Either have a system rsgroup where all the system tables are or just leave the
system tables in <code>default</code> rsgroup and have all user-space tables are in
non-<code>default</code> rsgroups.</p>
</div>
</div>
<div class="sect3">
<h4 id="_dead_nodes"><a class="anchor" href="#_dead_nodes"></a>171.1.2. Dead Nodes</h4>
<div class="paragraph">
<p>Yahoo! Have found it useful at their scale to keep a special rsgroup of dead or
questionable nodes; this is one means of keeping them out of the running until repair.</p>
</div>
<div class="paragraph">
<p>Be careful replacing dead nodes in an rsgroup. Ensure there are enough live nodes
before you start moving out the dead. Move in good live nodes first if you have to.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_troubleshooting"><a class="anchor" href="#_troubleshooting"></a>171.2. Troubleshooting</h3>
<div class="paragraph">
<p>Viewing the Master log will give you insight on rsgroup operation.</p>
</div>
<div class="paragraph">
<p>If it appears stuck, restart the Master process.</p>
</div>
</div>
<div class="sect2">
<h3 id="_remove_regionserver_grouping"><a class="anchor" href="#_remove_regionserver_grouping"></a>171.3. Remove RegionServer Grouping</h3>
<div class="paragraph">
<p>Removing RegionServer Grouping feature from a cluster on which it was enabled involves
more steps in addition to removing the relevant properties from <code>hbase-site.xml</code>. This is
to clean the RegionServer grouping related meta data so that if the feature is re-enabled
in the future, the old meta data will not affect the functioning of the cluster.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Move all tables in non-default rsgroups to <code>default</code> regionserver group</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>#Reassigning table t1 from non default group - hbase shell
hbase(main):005:0&gt; move_tables_rsgroup 'default',['t1']</pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Move all regionservers in non-default rsgroups to <code>default</code> regionserver group</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>#Reassigning all the servers in the non-default rsgroup to default - hbase shell
hbase(main):008:0&gt; move_servers_rsgroup 'default',['rs1.xxx.com:16206','rs2.xxx.com:16202','rs3.xxx.com:16204']</pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Remove all non-default rsgroups. <code>default</code> rsgroup created implicitly doesn&#8217;t have to be removed</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>#removing non default rsgroup - hbase shell
hbase(main):009:0&gt; remove_rsgroup 'group2'</pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Remove the changes made in <code>hbase-site.xml</code> and restart the cluster</p>
</li>
<li>
<p>Drop the table <code>hbase:rsgroup</code> from <code>hbase</code></p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>#Through hbase shell drop table hbase:rsgroup
hbase(main):001:0&gt; disable 'hbase:rsgroup'
0 row(s) in 2.6270 seconds
hbase(main):002:0&gt; drop 'hbase:rsgroup'
0 row(s) in 1.2730 seconds</pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>Remove znode <code>rsgroup</code> from the cluster ZooKeeper using zkCli.sh</p>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre>#From ZK remove the node /hbase/rsgroup through zkCli.sh
rmr /hbase/rsgroup</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_acl"><a class="anchor" href="#_acl"></a>171.4. ACL</h3>
<div class="paragraph">
<p>To enable ACL, add the following to your hbase-site.xml and restart your Master:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.security.authorization<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="normalizer"><a class="anchor" href="#normalizer"></a>172. Region Normalizer</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The Region Normalizer tries to make Regions all in a table about the same in size.
It does this by finding a rough average. Any region that is larger than twice this
size is split. Any region that is much smaller is merged into an adjacent region.
It is good to run the Normalizer on occasion on a down time after the cluster has
been running a while or say after a burst of activity such as a large delete.</p>
</div>
<div class="paragraph">
<p>(The bulk of the below detail was copied wholesale from the blog by Romil Choksi at
<a href="https://community.hortonworks.com/articles/54987/hbase-region-normalizer.html">HBase Region Normalizer</a>)</p>
</div>
<div class="paragraph">
<p>The Region Normalizer is feature available since HBase-1.2. It runs a set of
pre-calculated merge/split actions to resize regions that are either too
large or too small compared to the average region size for a given table. Region
Normalizer when invoked computes a normalization 'plan' for all of the tables in
HBase. System tables (such as hbase:meta, hbase:namespace, Phoenix system tables
etc) and user tables with normalization disabled are ignored while computing the
plan. For normalization enabled tables, normalization plan is carried out in
parallel across multiple tables.</p>
</div>
<div class="paragraph">
<p>Normalizer can be enabled or disabled globally for the entire cluster using the
‘normalizer_switch’ command in the HBase shell. Normalization can also be
controlled on a per table basis, which is disabled by default when a table is
created. Normalization for a table can be enabled or disabled by setting the
NORMALIZATION_ENABLED table attribute to true or false.</p>
</div>
<div class="paragraph">
<p>To check normalizer status and enable/disable normalizer</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">hbase(main):001:0&gt; normalizer_enabled
true
0 row(s) in 0.4870 seconds
hbase(main):002:0&gt; normalizer_switch false
true
0 row(s) in 0.0640 seconds
hbase(main):003:0&gt; normalizer_enabled
false
0 row(s) in 0.0120 seconds
hbase(main):004:0&gt; normalizer_switch true
false
0 row(s) in 0.0200 seconds
hbase(main):005:0&gt; normalizer_enabled
true
0 row(s) in 0.0090 seconds</code></pre>
</div>
</div>
<div class="paragraph">
<p>When enabled, Normalizer is invoked in the background every 5 mins (by default),
which can be configured using <code>hbase.normalization.period</code> in <code>hbase-site.xml</code>.
Normalizer can also be invoked manually/programmatically at will using HBase shell’s
<code>normalize</code> command. HBase by default uses <code>SimpleRegionNormalizer</code>, but users can
design their own normalizer as long as they implement the RegionNormalizer Interface.
Details about the logic used by <code>SimpleRegionNormalizer</code> to compute its normalization
plan can be found <a href="https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html">here</a>.</p>
</div>
<div class="paragraph">
<p>The below example shows a normalization plan being computed for an user table, and
merge action being taken as a result of the normalization plan computed by SimpleRegionNormalizer.</p>
</div>
<div class="paragraph">
<p>Consider an user table with some pre-split regions having 3 equally large regions
(about 100K rows) and 1 relatively small region (about 25K rows). Following is the
snippet from an hbase meta table scan showing each of the pre-split regions for
the user table.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>table_p8ddpd6q5z,,1469494305548.68b9892220865cb6048 column=info:regioninfo, timestamp=1469494306375, value={ENCODED =&gt; 68b9892220865cb604809c950d1adf48, NAME =&gt; 'table_p8ddpd6q5z,,1469494305548.68b989222 09c950d1adf48. 0865cb604809c950d1adf48.', STARTKEY =&gt; '', ENDKEY =&gt; '1'}
....
table_p8ddpd6q5z,1,1469494317178.867b77333bdc75a028 column=info:regioninfo, timestamp=1469494317848, value={ENCODED =&gt; 867b77333bdc75a028bb4c5e4b235f48, NAME =&gt; 'table_p8ddpd6q5z,1,1469494317178.867b7733 bb4c5e4b235f48. 3bdc75a028bb4c5e4b235f48.', STARTKEY =&gt; '1', ENDKEY =&gt; '3'}
....
table_p8ddpd6q5z,3,1469494328323.98f019a753425e7977 column=info:regioninfo, timestamp=1469494328486, value={ENCODED =&gt; 98f019a753425e7977ab8636e32deeeb, NAME =&gt; 'table_p8ddpd6q5z,3,1469494328323.98f019a7 ab8636e32deeeb. 53425e7977ab8636e32deeeb.', STARTKEY =&gt; '3', ENDKEY =&gt; '7'}
....
table_p8ddpd6q5z,7,1469494339662.94c64e748979ecbb16 column=info:regioninfo, timestamp=1469494339859, value={ENCODED =&gt; 94c64e748979ecbb166f6cc6550e25c6, NAME =&gt; 'table_p8ddpd6q5z,7,1469494339662.94c64e74 6f6cc6550e25c6. 8979ecbb166f6cc6550e25c6.', STARTKEY =&gt; '7', ENDKEY =&gt; '8'}
....
table_p8ddpd6q5z,8,1469494339662.6d2b3f5fd1595ab8e7 column=info:regioninfo, timestamp=1469494339859, value={ENCODED =&gt; 6d2b3f5fd1595ab8e7c031876057b1ee, NAME =&gt; 'table_p8ddpd6q5z,8,1469494339662.6d2b3f5f c031876057b1ee. d1595ab8e7c031876057b1ee.', STARTKEY =&gt; '8', ENDKEY =&gt; ''}</pre>
</div>
</div>
<div class="paragraph">
<p>Invoking the normalizer using ‘normalize’ int the HBase shell, the below log snippet
from HMaster log shows the normalization plan computed as per the logic defined for
SimpleRegionNormalizer. Since the total region size (in MB) for the adjacent smallest
regions in the table is less than the average region size, the normalizer computes a
plan to merge these two regions.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:namespace, as it's either system table or doesn't have auto
normalization turned on
2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:backup, as it's either system table or doesn't have auto normalization turned on
2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:meta, as it's either system table or doesn't have auto normalization turned on
2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: table_h2osxu3wat, as it's either system table or doesn't have autonormalization turned on
2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_p8ddpd6q5z, number of regions: 5
2016-07-26 07:08:26,929 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, total aggregated regions size: 12
2016-07-26 07:08:26,929 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, average region size: 2.4
2016-07-26 07:08:26,929 INFO [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, small region size: 0 plus its neighbor size: 0, less thanthe avg size 2.4, merging them
2016-07-26 07:08:26,971 INFO [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.MergeNormalizationPlan: Executing merging normalization plan: MergeNormalizationPlan{firstRegion={ENCODED=&gt; d51df2c58e9b525206b1325fd925a971, NAME =&gt; 'table_p8ddpd6q5z,,1469514755237.d51df2c58e9b525206b1325fd925a971.', STARTKEY =&gt; '', ENDKEY =&gt; '1'}, secondRegion={ENCODED =&gt; e69c6b25c7b9562d078d9ad3994f5330, NAME =&gt; 'table_p8ddpd6q5z,1,1469514767669.e69c6b25c7b9562d078d9ad3994f5330.',
STARTKEY =&gt; '1', ENDKEY =&gt; '3'}}</pre>
</div>
</div>
<div class="paragraph">
<p>Region normalizer as per it’s computed plan, merged the region with start key as ‘’
and end key as ‘1’, with another region having start key as ‘1’ and end key as ‘3’.
Now, that these regions have been merged we see a single new region with start key
as ‘’ and end key as ‘3’</p>
</div>
<div class="listingblock">
<div class="content">
<pre>table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeA, timestamp=1469516907431,
value=PBUF\x08\xA5\xD9\x9E\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x00"\x011(\x000\x00 ea74d246741ba. 8\x00
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeB, timestamp=1469516907431,
value=PBUF\x08\xB5\xBA\x9F\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x011"\x013(\x000\x0 ea74d246741ba. 08\x00
table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:regioninfo, timestamp=1469516907431, value={ENCODED =&gt; e06c9b83c4a252b130eea74d246741ba, NAME =&gt; 'table_p8ddpd6q5z,,1469516907210.e06c9b83c ea74d246741ba. 4a252b130eea74d246741ba.', STARTKEY =&gt; '', ENDKEY =&gt; '3'}
....
table_p8ddpd6q5z,3,1469514778736.bf024670a847c0adff column=info:regioninfo, timestamp=1469514779417, value={ENCODED =&gt; bf024670a847c0adffb74b2e13408b32, NAME =&gt; 'table_p8ddpd6q5z,3,1469514778736.bf024670 b74b2e13408b32. a847c0adffb74b2e13408b32.' STARTKEY =&gt; '3', ENDKEY =&gt; '7'}
....
table_p8ddpd6q5z,7,1469514790152.7c5a67bc755e649db2 column=info:regioninfo, timestamp=1469514790312, value={ENCODED =&gt; 7c5a67bc755e649db22f49af6270f1e1, NAME =&gt; 'table_p8ddpd6q5z,7,1469514790152.7c5a67bc 2f49af6270f1e1. 755e649db22f49af6270f1e1.', STARTKEY =&gt; '7', ENDKEY =&gt; '8'}
....
table_p8ddpd6q5z,8,1469514790152.58e7503cda69f98f47 column=info:regioninfo, timestamp=1469514790312, value={ENCODED =&gt; 58e7503cda69f98f4755178e74288c3a, NAME =&gt; 'table_p8ddpd6q5z,8,1469514790152.58e7503c 55178e74288c3a. da69f98f4755178e74288c3a.', STARTKEY =&gt; '8', ENDKEY =&gt; ''}</pre>
</div>
</div>
<div class="paragraph">
<p>A similar example can be seen for an user table with 3 smaller regions and 1
relatively large region. For this example, we have an user table with 1 large region containing 100K rows, and 3 relatively smaller regions with about 33K rows each. As seen from the normalization plan, since the larger region is more than twice the average region size it ends being split into two regions – one with start key as ‘1’ and end key as ‘154717’ and the other region with start key as '154717' and end key as ‘3’</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:backup, as it's either system table or doesn't have auto normalization turned on
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_p8ddpd6q5z, number of regions: 4
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, total aggregated regions size: 12
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, average region size: 3.0
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: No normalization needed, regions look good for table: table_p8ddpd6q5z
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_h2osxu3wat, number of regions: 5
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, total aggregated regions size: 7
2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, average region size: 1.4
2016-07-26 07:39:45,636 INFO [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, large region table_h2osxu3wat,1,1469515926544.27f2fdbb2b6612ea163eb6b40753c3db. has size 4, more than twice avg size, splitting
2016-07-26 07:39:45,640 INFO [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SplitNormalizationPlan: Executing splitting normalization plan: SplitNormalizationPlan{regionInfo={ENCODED =&gt; 27f2fdbb2b6612ea163eb6b40753c3db, NAME =&gt; 'table_h2osxu3wat,1,1469515926544.27f2fdbb2b6612ea163eb6b40753c3db.', STARTKEY =&gt; '1', ENDKEY =&gt; '3'}, splitPoint=null}
2016-07-26 07:39:45,656 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:namespace, as it's either system table or doesn't have auto normalization turned on
2016-07-26 07:39:45,656 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:meta, as it's either system table or doesn't
have auto normalization turned on …..…..….
2016-07-26 07:39:46,246 DEBUG [AM.ZK.Worker-pool2-t278] master.RegionStates: Onlined 54de97dae764b864504704c1c8d3674a on hbase-test-rc-5.openstacklocal,16020,1469419333913 {ENCODED =&gt; 54de97dae764b864504704c1c8d3674a, NAME =&gt; 'table_h2osxu3wat,1,1469518785661.54de97dae764b864504704c1c8d3674a.', STARTKEY =&gt; '1', ENDKEY =&gt; '154717'}
2016-07-26 07:39:46,246 INFO [AM.ZK.Worker-pool2-t278] master.RegionStates: Transition {d6b5625df331cfec84dce4f1122c567f state=SPLITTING_NEW, ts=1469518786246, server=hbase-test-rc-5.openstacklocal,16020,1469419333913} to {d6b5625df331cfec84dce4f1122c567f state=OPEN, ts=1469518786246,
server=hbase-test-rc-5.openstacklocal,16020,1469419333913}
2016-07-26 07:39:46,246 DEBUG [AM.ZK.Worker-pool2-t278] master.RegionStates: Onlined d6b5625df331cfec84dce4f1122c567f on hbase-test-rc-5.openstacklocal,16020,1469419333913 {ENCODED =&gt; d6b5625df331cfec84dce4f1122c567f, NAME =&gt; 'table_h2osxu3wat,154717,1469518785661.d6b5625df331cfec84dce4f1122c567f.', STARTKEY =&gt; '154717', ENDKEY =&gt; '3'}</pre>
</div>
</div>
</div>
</div>
<h1 id="developer" class="sect0"><a class="anchor" href="#developer"></a>Building and Developing Apache HBase</h1>
<div class="openblock partintro">
<div class="content">
This chapter contains information and guidelines for building and releasing HBase code and documentation.
Being familiar with these guidelines will help the HBase committers to use your contributions more easily.
</div>
</div>
<div class="sect1">
<h2 id="getting.involved"><a class="anchor" href="#getting.involved"></a>173. Getting Involved</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Apache HBase gets better only when people contribute! If you are looking to contribute to Apache HBase, look for <a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20HBASE%20AND%20labels%20in%20(beginner)%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened)">issues in JIRA tagged with the label 'beginner'</a>.
These are issues HBase contributors have deemed worthy but not of immediate priority and a good way to ramp on HBase internals.
See <a href="http://search-hadoop.com/m/DHED43re96">What label
is used for issues that are good on ramps for new contributors?</a> from the dev mailing list for background.</p>
</div>
<div class="paragraph">
<p>Before you get started submitting code to HBase, please refer to <a href="#developing">developing</a>.</p>
</div>
<div class="paragraph">
<p>As Apache HBase is an Apache Software Foundation project, see <a href="#asf">asf</a> for more information about how the ASF functions.</p>
</div>
<div class="sect2">
<h3 id="mailing.list"><a class="anchor" href="#mailing.list"></a>173.1. Mailing Lists</h3>
<div class="paragraph">
<p>Sign up for the dev-list and the user-list.
See the <a href="https://hbase.apache.org/mail-lists.html">mailing lists</a> page.
Posing questions - and helping to answer other people&#8217;s questions - is encouraged! There are varying levels of experience on both lists so patience and politeness are encouraged (and please stay on topic.)</p>
</div>
</div>
<div class="sect2">
<h3 id="slack"><a class="anchor" href="#slack"></a>173.2. Slack</h3>
<div class="paragraph">
<p>The Apache HBase project has its own link: <a href="http://apache-hbase.slack.com">Slack Channel</a> for real-time questions
and discussion. Mail <a href="mailto:dev@hbase.apache.org">dev@hbase.apache.org</a> to request an invite.</p>
</div>
</div>
<div class="sect2">
<h3 id="irc"><a class="anchor" href="#irc"></a>173.3. Internet Relay Chat (IRC)</h3>
<div class="paragraph">
<p>(NOTE: Our IRC channel seems to have been deprecated in favor of the above Slack channel)</p>
</div>
<div class="paragraph">
<p>For real-time questions and discussions, use the <code>#hbase</code> IRC channel on the <a href="https://freenode.net/">FreeNode</a> IRC network.
FreeNode offers a web-based client, but most people prefer a native client, and several clients are available for each operating system.</p>
</div>
</div>
<div class="sect2">
<h3 id="_jira"><a class="anchor" href="#_jira"></a>173.4. Jira</h3>
<div class="paragraph">
<p>Check for existing issues in <a href="https://issues.apache.org/jira/projects/HBASE/issues">Jira</a>.
If it&#8217;s either a new feature request, enhancement, or a bug, file a ticket.</p>
</div>
<div class="paragraph">
<p>We track multiple types of work in JIRA:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Bug: Something is broken in HBase itself.</p>
</li>
<li>
<p>Test: A test is needed, or a test is broken.</p>
</li>
<li>
<p>New feature: You have an idea for new functionality. It&#8217;s often best to bring
these up on the mailing lists first, and then write up a design specification
that you add to the feature request JIRA.</p>
</li>
<li>
<p>Improvement: A feature exists, but could be tweaked or augmented. It&#8217;s often
best to bring these up on the mailing lists first and have a discussion, then
summarize or link to the discussion if others seem interested in the
improvement.</p>
</li>
<li>
<p>Wish: This is like a new feature, but for something you may not have the
background to flesh out yourself.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Bugs and tests have the highest priority and should be actionable.</p>
</div>
<div class="sect3">
<h4 id="_guidelines_for_reporting_effective_issues"><a class="anchor" href="#_guidelines_for_reporting_effective_issues"></a>173.4.1. Guidelines for reporting effective issues</h4>
<div class="ulist">
<ul>
<li>
<p><strong>Search for duplicates</strong>: Your issue may have already been reported. Have a
look, realizing that someone else might have worded the summary differently.</p>
<div class="paragraph">
<p>Also search the mailing lists, which may have information about your problem
and how to work around it. Don&#8217;t file an issue for something that has already
been discussed and resolved on a mailing list, unless you strongly disagree
with the resolution <strong>and</strong> are willing to help take the issue forward.</p>
</div>
<div class="ulist">
<ul>
<li>
<p><strong>Discuss in public</strong>: Use the mailing lists to discuss what you&#8217;ve discovered
and see if there is something you&#8217;ve missed. Avoid using back channels, so
that you benefit from the experience and expertise of the project as a whole.</p>
</li>
<li>
<p><strong>Don&#8217;t file on behalf of others</strong>: You might not have all the context, and you
don&#8217;t have as much motivation to see it through as the person who is actually
experiencing the bug. It&#8217;s more helpful in the long term to encourage others
to file their own issues. Point them to this material and offer to help out
the first time or two.</p>
</li>
<li>
<p><strong>Write a good summary</strong>: A good summary includes information about the problem,
the impact on the user or developer, and the area of the code.</p>
<div class="ulist">
<ul>
<li>
<p>Good: <code>Address new license dependencies from hadoop3-alpha4</code></p>
</li>
<li>
<p>Room for improvement: <code>Canary is broken</code></p>
<div class="paragraph">
<p>If you write a bad title, someone else will rewrite it for you. This is time
they could have spent working on the issue instead.</p>
</div>
</li>
</ul>
</div>
</li>
<li>
<p><strong>Give context in the description</strong>: It can be good to think of this in multiple
parts:</p>
<div class="ulist">
<ul>
<li>
<p>What happens or doesn&#8217;t happen?</p>
</li>
<li>
<p>How does it impact you?</p>
</li>
<li>
<p>How can someone else reproduce it?</p>
</li>
<li>
<p>What would "fixed" look like?</p>
<div class="paragraph">
<p>You don&#8217;t need to know the answers for all of these, but give as much
information as you can. If you can provide technical information, such as a
Git commit SHA that you think might have caused the issue or a build failure
on builds.apache.org where you think the issue first showed up, share that
info.</p>
</div>
</li>
</ul>
</div>
</li>
<li>
<p><strong>Fill in all relevant fields</strong>: These fields help us filter, categorize, and
find things.</p>
</li>
<li>
<p><strong>One bug, one issue, one patch</strong>: To help with back-porting, don&#8217;t split issues
or fixes among multiple bugs.</p>
</li>
<li>
<p><strong>Add value if you can</strong>: Filing issues is great, even if you don&#8217;t know how to
fix them. But providing as much information as possible, being willing to
triage and answer questions, and being willing to test potential fixes is even
better! We want to fix your issue as quickly as you want it to be fixed.</p>
</li>
<li>
<p><strong>Don&#8217;t be upset if we don&#8217;t fix it</strong>: Time and resources are finite. In some
cases, we may not be able to (or might choose not to) fix an issue, especially
if it is an edge case or there is a workaround. Even if it doesn&#8217;t get fixed,
the JIRA is a public record of it, and will help others out if they run into
a similar issue in the future.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_working_on_an_issue"><a class="anchor" href="#_working_on_an_issue"></a>173.4.2. Working on an issue</h4>
<div class="paragraph">
<p>To check for existing issues which you can tackle as a beginner, search for <a href="https://issues.apache.org/jira/issues/?jql=project%20%3D%20HBASE%20AND%20labels%20in%20(beginner)%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened)">issues in JIRA tagged with the label 'beginner'</a>.</p>
</div>
<div class="ulist">
<div class="title">JIRA Priorites</div>
<ul>
<li>
<p><strong>Blocker</strong>: Should only be used if the issue WILL cause data loss or cluster instability reliably.</p>
</li>
<li>
<p><strong>Critical</strong>: The issue described can cause data loss or cluster instability in some cases.</p>
</li>
<li>
<p><strong>Major</strong>: Important but not tragic issues, like updates to the client API that will add a lot of much-needed functionality or significant bugs that need to be fixed but that don&#8217;t cause data loss.</p>
</li>
<li>
<p><strong>Minor</strong>: Useful enhancements and annoying but not damaging bugs.</p>
</li>
<li>
<p><strong>Trivial</strong>: Useful enhancements but generally cosmetic.</p>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 44. Code Blocks in Jira Comments</div>
<div class="content">
<div class="paragraph">
<p>A commonly used macro in Jira is {code}. Everything inside the tags is preformatted, as in this example.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">{code}
code snippet
{code}</code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="repos"><a class="anchor" href="#repos"></a>174. Apache HBase Repositories</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Apache HBase consists of multiple repositories which are hosted on <a href="https://gitbox.apache.org/">Apache GitBox</a>.
These are the following:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><a href="https://gitbox.apache.org/repos/asf?p=hbase.git">hbase</a> - main Apache HBase repository</p>
</li>
<li>
<p><a href="https://gitbox.apache.org/repos/asf?p=hbase-connectors.git">hbase-connectors</a> - connectors to Apache Kafka and Apache Spark</p>
</li>
<li>
<p><a href="https://gitbox.apache.org/repos/asf?p=hbase-operator-tools.git">hbase-operator-tools</a> - operability and supportability tools, such as <a href="#HBCK2">HBase <code>HBCK2</code></a></p>
</li>
<li>
<p><a href="https://gitbox.apache.org/repos/asf?p=hbase-site.git">hbase-site</a> - hbase.apache.org website</p>
</li>
<li>
<p><a href="https://gitbox.apache.org/repos/asf?p=hbase-thirdparty.git">hbase-thirdparty</a> - relocated versions of popular third-party libraries</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_ides"><a class="anchor" href="#_ides"></a>175. IDEs</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="eclipse"><a class="anchor" href="#eclipse"></a>175.1. Eclipse</h3>
<div class="sect3">
<h4 id="eclipse.code.formatting"><a class="anchor" href="#eclipse.code.formatting"></a>175.1.1. Code Formatting</h4>
<div class="paragraph">
<p>Under the <em>dev-support/</em> folder, you will find <em>hbase_eclipse_formatter.xml</em>.
We encourage you to have this formatter in place in eclipse when editing HBase code.</p>
</div>
<div class="paragraph">
<p>Go to <code>Preferences&#8594;Java&#8594;Code Style&#8594;Formatter&#8594;Import</code> to load the xml file.
Go to <code>Preferences&#8594;Java&#8594;Editor&#8594;Save Actions</code>, and make sure 'Format source code' and 'Format
edited lines' is selected.</p>
</div>
<div class="paragraph">
<p>In addition to the automatic formatting, make sure you follow the style guidelines explained in
<a href="#common.patch.feedback">common.patch.feedback</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.git.plugin"><a class="anchor" href="#eclipse.git.plugin"></a>175.1.2. Eclipse Git Plugin</h4>
<div class="paragraph">
<p>If you cloned the project via git, download and install the Git plugin (EGit). Attach to your local git repo (via the <span class="label">Git Repositories</span> window) and you&#8217;ll be able to see file revision history, generate patches, etc.</p>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.maven.setup"><a class="anchor" href="#eclipse.maven.setup"></a>175.1.3. HBase Project Setup in Eclipse using <code>m2eclipse</code></h4>
<div class="paragraph">
<p>The easiest way is to use the m2eclipse plugin for Eclipse.
Eclipse Indigo or newer includes m2eclipse, or you can download it from <a href="http://www.eclipse.org/m2e/" class="bare">http://www.eclipse.org/m2e/</a>. It provides Maven integration for Eclipse, and even lets you use the direct Maven commands from within Eclipse to compile and test your project.</p>
</div>
<div class="paragraph">
<p>To import the project, click and select the HBase root directory. <code>m2eclipse</code> locates all the hbase modules for you.</p>
</div>
<div class="paragraph">
<p>If you install m2eclipse and import HBase in your workspace, do the following to fix your eclipse Build Path.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Remove <em>target</em> folder</p>
</li>
<li>
<p>Add <em>target/generated-jamon</em> and <em>target/generated-sources/java</em> folders.</p>
</li>
<li>
<p>Remove from your Build Path the exclusions on the <em>src/main/resources</em> and <em>src/test/resources</em> to avoid error message in the console, such as the following:</p>
<div class="listingblock">
<div class="content">
<pre>Failed to execute goal
org.apache.maven.plugins:maven-antrun-plugin:1.6:run (default) on project hbase:
'An Ant BuildException has occurred: Replace: source file .../target/classes/hbase-default.xml
doesn't exist</pre>
</div>
</div>
<div class="paragraph">
<p>This will also reduce the eclipse build cycles and make your life easier when developing.</p>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.commandline"><a class="anchor" href="#eclipse.commandline"></a>175.1.4. HBase Project Setup in Eclipse Using the Command Line</h4>
<div class="paragraph">
<p>Instead of using <code>m2eclipse</code>, you can generate the Eclipse files from the command line.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>First, run the following command, which builds HBase.
You only need to do this once.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean install -DskipTests</code></pre>
</div>
</div>
</li>
<li>
<p>Close Eclipse, and execute the following command from the terminal, in your local HBase project directory, to generate new <em>.project</em> and <em>.classpath</em> files.</p>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn eclipse:eclipse</code></pre>
</div>
</div>
</li>
<li>
<p>Reopen Eclipse and import the <em>.project</em> file in the HBase directory to a workspace.</p>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.maven.class"><a class="anchor" href="#eclipse.maven.class"></a>175.1.5. Maven Classpath Variable</h4>
<div class="paragraph">
<p>The <code>$M2_REPO</code> classpath variable needs to be set up for the project.
This needs to be set to your local Maven repository, which is usually <em>~/.m2/repository</em></p>
</div>
<div class="paragraph">
<p>If this classpath variable is not configured, you will see compile errors in Eclipse like this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Description Resource Path Location Type
The project cannot be built until build path errors are resolved hbase Unknown Java Problem
Unbound classpath variable: 'M2_REPO/asm/asm/3.1/asm-3.1.jar' in project 'hbase' hbase Build path Build Path Problem
Unbound classpath variable: 'M2_REPO/com/google/guava/guava/r09/guava-r09.jar' in project 'hbase' hbase Build path Build Path Problem
Unbound classpath variable: 'M2_REPO/com/google/protobuf/protobuf-java/2.3.0/protobuf-java-2.3.0.jar' in project 'hbase' hbase Build path Build Path Problem Unbound classpath variable:</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.issues"><a class="anchor" href="#eclipse.issues"></a>175.1.6. Eclipse Known Issues</h4>
<div class="paragraph">
<p>Eclipse will currently complain about <em>Bytes.java</em>.
It is not possible to turn these errors off.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Description Resource Path Location Type
Access restriction: The method arrayBaseOffset(Class) from the type Unsafe is not accessible due to restriction on required library /System/Library/Java/JavaVirtualMachines/1.6.0.jdk/Contents/Classes/classes.jar Bytes.java /hbase/src/main/java/org/apache/hadoop/hbase/util line 1061 Java Problem
Access restriction: The method arrayIndexScale(Class) from the type Unsafe is not accessible due to restriction on required library /System/Library/Java/JavaVirtualMachines/1.6.0.jdk/Contents/Classes/classes.jar Bytes.java /hbase/src/main/java/org/apache/hadoop/hbase/util line 1064 Java Problem
Access restriction: The method getLong(Object, long) from the type Unsafe is not accessible due to restriction on required library /System/Library/Java/JavaVirtualMachines/1.6.0.jdk/Contents/Classes/classes.jar Bytes.java /hbase/src/main/java/org/apache/hadoop/hbase/util line 1111 Java Problem</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="eclipse.more"><a class="anchor" href="#eclipse.more"></a>175.1.7. Eclipse - More Information</h4>
<div class="paragraph">
<p>For additional information on setting up Eclipse for HBase development on Windows, see <a href="http://michaelmorello.blogspot.com/2011/09/hbase-subversion-eclipse-windows.html">Michael Morello&#8217;s blog</a> on the topic.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_intellij_idea"><a class="anchor" href="#_intellij_idea"></a>175.2. IntelliJ IDEA</h3>
<div class="paragraph">
<p>You can set up IntelliJ IDEA for similar functionality as Eclipse.
Follow these steps.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Select</p>
</li>
<li>
<p>You do not need to select a profile.
Be sure <span class="label">Maven project
required</span> is selected, and click <b class="button">Next</b>.</p>
</li>
<li>
<p>Select the location for the JDK.</p>
</li>
</ol>
</div>
<div class="paragraph">
<div class="title">Using the HBase Formatter in IntelliJ IDEA</div>
<p>Using the Eclipse Code Formatter plugin for IntelliJ IDEA, you can import the HBase code formatter described in <a href="#eclipse.code.formatting">eclipse.code.formatting</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="_other_ides"><a class="anchor" href="#_other_ides"></a>175.3. Other IDEs</h3>
<div class="paragraph">
<p>It would be useful to mirror the <a href="#eclipse">eclipse</a> set-up instructions for other IDEs.
If you would like to assist, please have a look at <a href="https://issues.apache.org/jira/browse/HBASE-11704">HBASE-11704</a>.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="build"><a class="anchor" href="#build"></a>176. Building Apache HBase</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="build.basic"><a class="anchor" href="#build.basic"></a>176.1. Basic Compile</h3>
<div class="paragraph">
<p>HBase is compiled using Maven.
You must use at least Maven 3.0.4.
To check your Maven version, run the command mvn -version.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">JDK Version Requirements</div>
<div class="paragraph">
<p>Starting with HBase 1.0 you must use Java 7 or later to build from source code.
See <a href="#java">java</a> for more complete information about supported JDK versions.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect3">
<h4 id="maven.build.commands"><a class="anchor" href="#maven.build.commands"></a>176.1.1. Maven Build Commands</h4>
<div class="paragraph">
<p>All commands are executed from the local HBase project directory.</p>
</div>
<div class="sect4">
<h5 id="_package"><a class="anchor" href="#_package"></a>Package</h5>
<div class="paragraph">
<p>The simplest command to compile HBase from its java source code is to use the <code>package</code> target, which builds JARs with the compiled files.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn package -DskipTests</code></pre>
</div>
</div>
<div class="paragraph">
<p>Or, to clean up before compiling:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean package -DskipTests</code></pre>
</div>
</div>
<div class="paragraph">
<p>With Eclipse set up as explained above in <a href="#eclipse">eclipse</a>, you can also use the <span class="menu">Build</span> command in Eclipse.
To create the full installable HBase package takes a little bit more work, so read on.</p>
</div>
</div>
<div class="sect4">
<h5 id="maven.build.commands.compile"><a class="anchor" href="#maven.build.commands.compile"></a>Compile</h5>
<div class="paragraph">
<p>The <code>compile</code> target does not create the JARs with the compiled files.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn compile</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean compile</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="_install"><a class="anchor" href="#_install"></a>Install</h5>
<div class="paragraph">
<p>To install the JARs in your <em>~/.m2/</em> directory, use the <code>install</code> target.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn install</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean install</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean install -DskipTests</code></pre>
</div>
</div>
</div>
</div>
<div class="sect3">
<h4 id="maven.build.commands.unitall"><a class="anchor" href="#maven.build.commands.unitall"></a>176.1.2. Running all or individual Unit Tests</h4>
<div class="paragraph">
<p>See the <a href="#hbase.unittests.cmds">hbase.unittests.cmds</a> section in <a href="#hbase.unittests">hbase.unittests</a></p>
</div>
</div>
<div class="sect3">
<h4 id="maven.build.hadoop"><a class="anchor" href="#maven.build.hadoop"></a>176.1.3. Building against various hadoop versions.</h4>
<div class="paragraph">
<p>HBase supports building against Apache Hadoop versions: 2.y and 3.y (early release artifacts). By default we build against Hadoop 2.x.</p>
</div>
<div class="paragraph">
<p>To build against a specific release from the Hadoop 2.y line, set e.g. <code>-Dhadoop-two.version=2.6.3</code>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn -Dhadoop-two.version=2.6.3 ...</code></pre>
</div>
</div>
<div class="paragraph">
<p>To change the major release line of Hadoop we build against, add a hadoop.profile property when you invoke mvn:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn -Dhadoop.profile=3.0 ...</code></pre>
</div>
</div>
<div class="paragraph">
<p>The above will build against whatever explicit hadoop 3.y version we have in our <em>pom.xml</em> as our '3.0' version.
Tests may not all pass so you may need to pass <code>-DskipTests</code> unless you are inclined to fix the failing tests.</p>
</div>
<div class="paragraph">
<p>To pick a particular Hadoop 3.y release, you&#8217;d set hadoop-three.version property e.g. <code>-Dhadoop-three.version=3.0.0</code>.</p>
</div>
</div>
<div class="sect3">
<h4 id="build.protobuf"><a class="anchor" href="#build.protobuf"></a>176.1.4. Build Protobuf</h4>
<div class="paragraph">
<p>You may need to change the protobuf definitions that reside in the <em>hbase-protocol</em> module or other modules.</p>
</div>
<div class="paragraph">
<p>Previous to hbase-2.0.0, protobuf definition files were sprinkled across all hbase modules but now all
to do with protobuf must reside in the hbase-protocol module; we are trying to contain our protobuf
use so we can freely change versions without upsetting any downstream project use of protobuf.</p>
</div>
<div class="paragraph">
<p>The protobuf files are located in <em>hbase-protocol/src/main/protobuf</em>.
For the change to be effective, you will need to regenerate the classes.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn package -pl hbase-protocol -am</code></pre>
</div>
</div>
<div class="paragraph">
<p>Similarly, protobuf definitions for internal use are located in the <em>hbase-protocol-shaded</em> module.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn package -pl hbase-protocol-shaded -am</code></pre>
</div>
</div>
<div class="paragraph">
<p>Typically, protobuf code generation is done using the native <code>protoc</code> binary. In our build we use a maven plugin for
convenience; however, the plugin may not be able to retrieve appropriate binaries for all platforms. If you find yourself
on a platform where protoc fails, you will have to compile protoc from source, and run it independent of our maven build.
You can disable the inline code generation by specifying <code>-Dprotoc.skip</code> in your maven arguments, allowing your build to proceed further.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
If you need to manually generate your protobuf files, you should not use <code>clean</code> in subsequent maven calls, as that will delete the newly generated files.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Read the <em>hbase-protocol/README.txt</em> for more details</p>
</div>
</div>
<div class="sect3">
<h4 id="build.thrift"><a class="anchor" href="#build.thrift"></a>176.1.5. Build Thrift</h4>
<div class="paragraph">
<p>You may need to change the thrift definitions that reside in the <em>hbase-thrift</em> module or other modules.</p>
</div>
<div class="paragraph">
<p>The thrift files are located in <em>hbase-thrift/src/main/resources</em>.
For the change to be effective, you will need to regenerate the classes.
You can use maven profile <code>compile-thrift</code> to do this.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn compile -Pcompile-thrift</code></pre>
</div>
</div>
<div class="paragraph">
<p>You may also want to define <code>thrift.path</code> for the thrift binary, using the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> mvn compile -Pcompile-thrift -Dthrift.path=/opt/local/bin/thrift</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_build_a_tarball"><a class="anchor" href="#_build_a_tarball"></a>176.1.6. Build a Tarball</h4>
<div class="paragraph">
<p>You can build a tarball without going through the release process described in <a href="#releasing">releasing</a>, by running the following command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn -DskipTests clean install &amp;&amp; mvn -DskipTests package assembly:single</pre>
</div>
</div>
<div class="paragraph">
<p>The distribution tarball is built in <em>hbase-assembly/target/hbase-&lt;version&gt;-bin.tar.gz</em>.</p>
</div>
<div class="paragraph">
<p>You can install or deploy the tarball by having the assembly:single goal before install or deploy in the maven command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn -DskipTests package assembly:single install</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn -DskipTests package assembly:single deploy</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="build.gotchas"><a class="anchor" href="#build.gotchas"></a>176.1.7. Build Gotchas</h4>
<div class="sect4">
<h5 id="_maven_site_failure"><a class="anchor" href="#_maven_site_failure"></a>Maven Site failure</h5>
<div class="paragraph">
<p>If you see <code>Unable to find resource 'VM_global_library.vm'</code>, ignore it.
It&#8217;s not an error.
It is <a href="https://issues.apache.org/jira/browse/MSITE-286">officially ugly</a> though.</p>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="releasing"><a class="anchor" href="#releasing"></a>177. Releasing Apache HBase</h2>
<div class="sectionbody">
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Building against HBase 1.x</div>
<div class="paragraph">
<p>HBase 1.x requires Java 7 to build.
See <a href="#java">java</a> for Java requirements per HBase release.</p>
</div>
</td>
</tr>
</table>
</div>
<div id="maven.settings.xml" class="exampleblock">
<div class="title">Example 45. Example <em>~/.m2/settings.xml</em> File</div>
<div class="content">
<div class="paragraph">
<p>Publishing to maven requires you sign the artifacts you want to upload.
For the build to sign them for you, you a properly configured <em>settings.xml</em> in your local repository under <em>.m2</em>, such as the following.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;settings</span> <span class="attribute-name">xmlns</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">http://maven.apache.org/SETTINGS/1.0.0</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">xmlns:xsi</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">http://www.w3.org/2001/XMLSchema-instance</span><span class="delimiter">&quot;</span></span>
<span class="attribute-name">xsi:schemaLocation</span>=<span class="string"><span class="delimiter">&quot;</span><span class="content">http://maven.apache.org/SETTINGS/1.0.0</span>
<span class="content">http://maven.apache.org/xsd/settings-1.0.0.xsd</span><span class="delimiter">&quot;</span></span><span class="tag">&gt;</span>
<span class="tag">&lt;servers&gt;</span>
<span class="error">&lt;</span>!- To publish a snapshot of some part of Maven --<span class="error">&gt;</span>
<span class="tag">&lt;server&gt;</span>
<span class="tag">&lt;id&gt;</span>apache.snapshots.https<span class="tag">&lt;/id&gt;</span>
<span class="tag">&lt;username&gt;</span>YOUR_APACHE_ID
<span class="tag">&lt;/username&gt;</span>
<span class="tag">&lt;password&gt;</span>YOUR_APACHE_PASSWORD
<span class="tag">&lt;/password&gt;</span>
<span class="tag">&lt;/server&gt;</span>
<span class="comment">&lt;!-- To publish a website using Maven --&gt;</span>
<span class="comment">&lt;!-- To stage a release of some part of Maven --&gt;</span>
<span class="tag">&lt;server&gt;</span>
<span class="tag">&lt;id&gt;</span>apache.releases.https<span class="tag">&lt;/id&gt;</span>
<span class="tag">&lt;username&gt;</span>YOUR_APACHE_ID
<span class="tag">&lt;/username&gt;</span>
<span class="tag">&lt;password&gt;</span>YOUR_APACHE_PASSWORD
<span class="tag">&lt;/password&gt;</span>
<span class="tag">&lt;/server&gt;</span>
<span class="tag">&lt;/servers&gt;</span>
<span class="tag">&lt;profiles&gt;</span>
<span class="tag">&lt;profile&gt;</span>
<span class="tag">&lt;id&gt;</span>apache-release<span class="tag">&lt;/id&gt;</span>
<span class="tag">&lt;properties&gt;</span>
<span class="tag">&lt;gpg.keyname&gt;</span>YOUR_KEYNAME<span class="tag">&lt;/gpg.keyname&gt;</span>
<span class="comment">&lt;!--Keyname is something like this ... 00A5F21E... do gpg --list-keys to find it--&gt;</span>
<span class="tag">&lt;gpg.passphrase&gt;</span>YOUR_KEY_PASSWORD
<span class="tag">&lt;/gpg.passphrase&gt;</span>
<span class="tag">&lt;/properties&gt;</span>
<span class="tag">&lt;/profile&gt;</span>
<span class="tag">&lt;/profiles&gt;</span>
<span class="tag">&lt;/settings&gt;</span></code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="maven.release"><a class="anchor" href="#maven.release"></a>177.1. Making a Release Candidate</h3>
<div class="paragraph">
<p>Only committers can make releases of hbase artifacts.</p>
</div>
<div class="paragraph">
<div class="title">Before You Begin</div>
<p>Check to be sure recent builds have been passing for the branch from where you
are going to take your release. You should also have tried recent branch tips
out on a cluster under load, perhaps by running the <code>hbase-it</code> integration test
suite for a few hours to 'burn in' the near-candidate bits.</p>
</div>
<div class="paragraph">
<p>You will need a published signing key added to the hbase
<a href="https://dist.apache.org/repos/dist/release/hbase/KEYS">KEYS</a> file.
(For how to add a KEY, see <em>Step 1.</em> in <a href="https://cwiki.apache.org/confluence/display/HADOOP2/HowToRelease">How To Release</a>,
the Hadoop version of this document).</p>
</div>
<div class="paragraph">
<p>Next make sure JIRA is properly primed, that all issues targeted against
the prospective release have been resolved and are present in git on the
particular branch. If any outstanding issues, move them out of the release by
adjusting the fix version to remove this pending release as a target.
Any JIRA with a fix version that matches the release candidate
target release will be included in the generated <em>CHANGES.md/RELEASENOTES.md</em>
files that ship with the release so make sure JIRA is correct before you begin.</p>
</div>
<div class="paragraph">
<p>After doing the above, you can move to the manufacture of an RC.
Building an RC is involved. We&#8217;ve tried to script it. In the next section
we describe the script. It is followed by a description of the steps
involved which the script automates.</p>
</div>
<div class="sect3">
<h4 id="do-release-docker.sh"><a class="anchor" href="#do-release-docker.sh"></a>177.1.1. Release Candidate Generating Script</h4>
<div class="paragraph">
<p>The <em>dev-support/create-release/do-release-docker.sh</em> Release Candidate (RC)
Generating script is maintained in the master branch but can generate RCs
for any 2.x+ branch (The script does not work against branch-1). Check out
and update the master branch when making RCs.</p>
</div>
<div class="paragraph">
<p>The script builds in a Docker container to ensure we have a consistent
environment building. It will ask you for passwords for apache and for your
gpg signing key so it can sign and commit on your behalf. The passwords
are passed to gpg-agent in the container and purged along with the container
when the build is done.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="paragraph">
<p><em>dev-support/create-release/do-release-docker.sh</em> supercedes the previous
<em>dev-support/make_rc.sh</em> script. It is more comprehensive automating all
steps, rather than a portion, building a RC.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>The script will:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Set version to the release version</p>
</li>
<li>
<p>Updates RELEASENOTES.md and CHANGES.md</p>
</li>
<li>
<p>Tag the RC</p>
</li>
<li>
<p>Set version to next SNAPSHOT version.</p>
</li>
<li>
<p>Builds, signs, and hashes all artifacts.</p>
</li>
<li>
<p>Generates the api compatibility report</p>
</li>
<li>
<p>Pushes release tgzs to the dev dir in a apache dist.</p>
</li>
<li>
<p>Pushes to repository.apache.org staging.</p>
</li>
<li>
<p>Creates vote email template.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The RC building script is <em>dev-support/create-release/do-release-docker.sh</em>.
Pass <em>-h</em> to <em>dev-support/create-release/do-release-docker.sh</em> to
see available options:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Usage: <span class="keyword">do</span>-release-docker.sh [options]
This script runs the release scripts inside a docker image.
Options:
-d [path] required. working directory. output will be written to <span class="string"><span class="delimiter">&quot;</span><span class="content">output</span><span class="delimiter">&quot;</span></span> in here.
-n dry run mode. Checks and local builds, but does not upload anything.
-t [tag] tag <span class="keyword">for</span> the hbase-rm docker image to use <span class="keyword">for</span> building (<span class="keyword">default</span>: <span class="string"><span class="delimiter">&quot;</span><span class="content">latest</span><span class="delimiter">&quot;</span></span>).
-j [path] path to local JDK installation to use building. By <span class="keyword">default</span> the script will
use openjdk8 installed in the docker image.
-s [step] runs a single step of the process; valid steps are: tag, build, publish. if
none specified, runs tag, then build, and then publish.</code></pre>
</div>
</div>
<div class="paragraph">
<p>Running the below command will do all steps above using the
'rm' working directory under Downloads as workspace:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> <span class="error">$</span> ./dev-support/create-release/<span class="keyword">do</span>-release-docker.sh -d ~/Downloads/rm</code></pre>
</div>
</div>
<div class="paragraph">
<p>The script will ask you a set of questions about the release version
and branch, the version to generate the compatibility report against,
and so on, before it starts executing (If you set the appropriate
environment variables, the script will skip asking you questions&#8201;&#8212;&#8201;which can come in handy if you end up having to re-run the script
multiple times).</p>
</div>
<div class="paragraph">
<p>On branch 2.1, a Release Candidate (RC) creation can take many hours
(~8 hours) so run your build on a machine you know will be
around for this swath of time. Start the build inside a <em>screen</em>
or <em>tmux</em> session in case you become disconnected from your
build box.</p>
</div>
<div class="paragraph">
<p>The build is made of three stages: tag, build, and
publish. If the script fails, you may resort to 'fixing' the
failure manually and then asking the script to run the
subsequent stage rather than start over.</p>
</div>
<div class="paragraph">
<p>When the scripts run, they use the passed working directory.
Under the working directory is an <em>output</em> dir. In here is
where the checkouts go, where we build up the <em>svn</em> directory
to <em>svn</em> commit to <em>apache/dist/dev</em>, etc. Each step also
dumps a log file in here: e.g. <em>tag.log</em> for the tagging
step and <em>build.log</em> for building.</p>
</div>
<div class="paragraph">
<p>The <em>tagging</em> step will checkout hbase, set the version number
in all the poms – e.g. if branch-2.0 is at 2.0.6-SNAPSHOT
and you are making a 2.0.5 RC, it will set the versions in
all poms to 2.0.5 – appropriately. It then generate CHANGES.md
and RELEASENOTES.md by checking out yetus and then
calling its generator scripts. It then commits the poms with
their new versions along with the changed CHANGES.md and
RELEASENOTES.md, tags, and pushes up all changes to the
apache repo.</p>
</div>
<div class="paragraph">
<p>The <em>build</em> step will checkout hbase, build all including
javadoc and doc (javadoc takes the bulk of the time – 4 hours plus),
run assemblies to produce src and bin tarballs, sign and hash it
all, and then make a dir under apache dist dev named for the RC.
It will copy all artifacts in here including top-level CHANGES.md
and RELEASENOTES.md. It will generate api diff docs and put them
into this RC dir too. When done, it commits the svn RC.</p>
</div>
<div class="paragraph">
<p>The publish step will checkout hbase, build, and then copy up all
artifacts to repository.apache.org (signed and hashed). When done,
it will dump out an email template with all the correct links in place.</p>
</div>
<div class="paragraph">
<p>Check the artifacts pushed to the dev distribution directory and up
in repository.apache.org. If all looks good, check the generated
email and send to the dev list.</p>
</div>
<div class="paragraph">
<p>Under the create-release dir, scripts should make some sense:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">do</span>-release-docker.sh <span class="error">#</span> Main entrance.
do-release.sh . <span class="error">#</span> More checks. Not usable really other than by setting env variables before running it.
release-tag.sh <span class="error">#</span> Does tagging steps.
release-build.sh . <span class="error">#</span> Does the build and publish step.
release-util.sh <span class="error">#</span> Utility used by all of the above.
vote.tmpl <span class="error">#</span> Template <span class="keyword">for</span> email to send out.
hbase-rm <span class="error">#</span> Has docker image we use.</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the RC fails, the script will do the right thing when it comes
to edit of the <em>CHANGES.md</em> and <em>RELEASENOTES.md</em> removing the old
and updating the files with the updated content (No harm verifying
though).</p>
</div>
<div class="paragraph">
<p>One trick for checking stuff especially in utility is to do as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> source release-util.sh ; generate_api_report ../../ rel/<span class="float">2.1</span><span class="float">.3</span> <span class="float">2.14</span>RC1</code></pre>
</div>
</div>
<div class="paragraph">
<p>i.e. source the release-util.sh script and then run one of its functions
passing args. Helped debugging stuff.</p>
</div>
</div>
<div class="sect3">
<h4 id="rc_procedure"><a class="anchor" href="#rc_procedure"></a>177.1.2. Release Candidate Procedure</h4>
<div class="paragraph">
<p>Here we describe the steps involved generating a Release Candidate, the steps
automated by the script described in the previous section.</p>
</div>
<div class="paragraph">
<p>The process below makes use of various tools, mainly <em>git</em> and <em>maven</em>.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Specifying the Heap Space for Maven</div>
<div class="paragraph">
<p>You may run into OutOfMemoryErrors building, particularly building the site and
documentation. Up the heap for Maven by setting the <code>MAVEN_OPTS</code> variable.
You can prefix the variable to the Maven command, as in the following example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>MAVEN_OPTS="-Xmx4g -XX:MaxPermSize=256m" mvn package</pre>
</div>
</div>
<div class="paragraph">
<p>You could also set this in an environment variable or alias in your shell.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="sect4">
<h5 id="_update_the_em_changes_md_em_and_em_releasenotes_md_em_files_and_the_pom_files"><a class="anchor" href="#_update_the_em_changes_md_em_and_em_releasenotes_md_em_files_and_the_pom_files"></a>Update the <em>CHANGES.md</em> and <em>RELEASENOTES.md</em> files and the POM files.</h5>
<div class="paragraph">
<p>Update <em>CHANGES.md</em> with the changes since the last release. Be careful with where you put
headings and license. Respect the instructions and warning you find in current
<em>CHANGES.md</em> and <em>RELEASENOTES.md</em> since these two files are processed by tooling that is
looking for particular string sequences. See <a href="https://issues.apache.org/jira/browse/HBASE-21399">HBASE-21399</a>
for description on how to make use of yetus generating additions to
<em>CHANGES.md</em> and <em>RELEASENOTES.md</em> (RECOMMENDED!). Adding JIRA fixes, make sure the
URL to the JIRA points to the proper location which lists fixes for this release.</p>
</div>
<div class="paragraph">
<p>Next, adjust the version in all the POM files appropriately.
If you are making a release candidate, you must remove the <code>-SNAPSHOT</code> label from all versions
in all pom.xml files.
If you are running this receipe to publish a snapshot, you must keep the <code>-SNAPSHOT</code> suffix on the hbase version.
The <a href="http://www.mojohaus.org/versions-maven-plugin/">Versions Maven Plugin</a> can be of use here.
To set a version in all the many poms of the hbase multi-module project, use a command like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn clean org.codehaus.mojo:versions-maven-plugin:2.5:set -DnewVersion=2.1.0-SNAPSHOT </code></pre>
</div>
</div>
<div class="paragraph">
<p>Make sure all versions in poms are changed! Checkin the <em>CHANGES.md</em>, <em>RELEASENOTES.md</em>, and
any maven version changes.</p>
</div>
</div>
<div class="sect4">
<h5 id="_update_the_documentation"><a class="anchor" href="#_update_the_documentation"></a>Update the documentation.</h5>
<div class="paragraph">
<p>Update the documentation under <em>src/main/asciidoc</em>.
This usually involves copying the latest from master branch and making version-particular
adjustments to suit this release candidate version. Commit your changes.</p>
</div>
</div>
<div class="sect4">
<h5 id="_clean_the_checkout_dir"><a class="anchor" href="#_clean_the_checkout_dir"></a>Clean the checkout dir</h5>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn clean
$ git clean -f -x -d</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="_run_apache_rat"><a class="anchor" href="#_run_apache_rat"></a>Run Apache-Rat</h5>
<div class="paragraph">
<p>Check licenses are good</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn apache-rat:check</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the above fails, check the rat log.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ grep 'Rat check' patchprocess/mvn_apache_rat.log</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="_create_a_release_tag"><a class="anchor" href="#_create_a_release_tag"></a>Create a release tag.</h5>
<div class="paragraph">
<p>Presuming you have run basic tests, the rat check, passes and all is
looking good, now is the time to tag the release candidate (You
always remove the tag if you need to redo). To tag, do
what follows substituting in the version appropriate to your build.
All tags should be signed tags; i.e. pass the <em>-s</em> option (See
<a href="http://https://git-scm.com/book/id/v2/Git-Tools-Signing-Your-Work">Signing Your Work</a>
for how to set up your git environment for signing).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ git tag -s 2.0.0-alpha4-RC0 -m &quot;Tagging the 2.0.0-alpha4 first Releae Candidate (Candidates start at zero)&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Or, if you are making a release, tags should have a <em>rel/</em> prefix to ensure
they are preserved in the Apache repo as in:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">+$ git tag -s rel/2.0.0-alpha4 -m &quot;Tagging the 2.0.0-alpha4 Release&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Push the (specific) tag (only) so others have access.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ git push origin 2.0.0-alpha4-RC0</code></pre>
</div>
</div>
<div class="paragraph">
<p>For how to delete tags, see
<a href="http://www.manikrathee.com/how-to-delete-a-tag-in-git.html">How to Delete a Tag</a>. Covers
deleting tags that have not yet been pushed to the remote Apache
repo as well as delete of tags pushed to Apache.</p>
</div>
</div>
<div class="sect4">
<h5 id="_build_the_source_tarball"><a class="anchor" href="#_build_the_source_tarball"></a>Build the source tarball.</h5>
<div class="paragraph">
<p>Now, build the source tarball. Lets presume we are building the source
tarball for the tag <em>2.0.0-alpha4-RC0</em> into <em>/tmp/hbase-2.0.0-alpha4-RC0/</em>
(This step requires that the mvn and git clean steps described above have just been done).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ git archive --format=tar.gz --output=&quot;/tmp/hbase-2.0.0-alpha4-RC0/hbase-2.0.0-alpha4-src.tar.gz&quot; --prefix=&quot;hbase-2.0.0-alpha4/&quot; $git_tag</code></pre>
</div>
</div>
<div class="paragraph">
<p>Above we generate the hbase-2.0.0-alpha4-src.tar.gz tarball into the
<em>/tmp/hbase-2.0.0-alpha4-RC0</em> build output directory (We don&#8217;t want the <em>RC0</em> in the name or prefix.
These bits are currently a release candidate but if the VOTE passes, they will become the release so we do not taint
the artifact names with <em>RCX</em>).</p>
</div>
</div>
<div class="sect4">
<h5 id="_build_the_binary_tarball"><a class="anchor" href="#_build_the_binary_tarball"></a>Build the binary tarball.</h5>
<div class="paragraph">
<p>Next, build the binary tarball. Add the <code>-Prelease</code> profile when building.
It runs the license apache-rat check among other rules that help ensure
all is wholesome. Do it in two steps.</p>
</div>
<div class="paragraph">
<p>First install into the local repository</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn clean install -DskipTests -Prelease</code></pre>
</div>
</div>
<div class="paragraph">
<p>Next, generate documentation and assemble the tarball. Be warned,
this next step can take a good while, a couple of hours generating site
documentation.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn install -DskipTests site assembly:single -Prelease</code></pre>
</div>
</div>
<div class="paragraph">
<p>Otherwise, the build complains that hbase modules are not in the maven repository
when you try to do it all in one step, especially on a fresh repository.
It seems that you need the install goal in both steps.</p>
</div>
<div class="paragraph">
<p>Extract the generated tarball&#8201;&#8212;&#8201;you&#8217;ll find it under
<em>hbase-assembly/target</em> and check it out.
Look at the documentation, see if it runs, etc.
If good, copy the tarball beside the source tarball in the
build output directory.</p>
</div>
</div>
<div class="sect4">
<h5 id="_deploy_to_the_maven_repository"><a class="anchor" href="#_deploy_to_the_maven_repository"></a>Deploy to the Maven Repository.</h5>
<div class="paragraph">
<p>Next, deploy HBase to the Apache Maven repository. Add the
apache-release` profile when running the <code>mvn deploy</code> command.
This profile comes from the Apache parent pom referenced by our pom files.
It does signing of your artifacts published to Maven, as long as the
<em>settings.xml</em> is configured correctly, as described in <a href="#maven.settings.xml">Example <em>~/.m2/settings.xml</em> File</a>.
This step depends on the local repository having been populate
by the just-previous bin tarball build.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ mvn deploy -DskipTests -Papache-release -Prelease</code></pre>
</div>
</div>
<div class="paragraph">
<p>This command copies all artifacts up to a temporary staging Apache mvn repository in an 'open' state.
More work needs to be done on these maven artifacts to make them generally available.</p>
</div>
<div class="paragraph">
<p>We do not release HBase tarball to the Apache Maven repository. To avoid deploying the tarball, do not
include the <code>assembly:single</code> goal in your <code>mvn deploy</code> command. Check the deployed artifacts as described in the next section.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">make_rc.sh</div>
<div class="paragraph">
<p>If you ran the old <em>dev-support/make_rc.sh</em> script, this is as far as it takes you.
To finish the release, take up the script from here on out.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
<div class="sect4">
<h5 id="_make_the_release_candidate_available"><a class="anchor" href="#_make_the_release_candidate_available"></a>Make the Release Candidate available.</h5>
<div class="paragraph">
<p>The artifacts are in the maven repository in the staging area in the 'open' state.
While in this 'open' state you can check out what you&#8217;ve published to make sure all is good.
To do this, log in to Apache&#8217;s Nexus at <a href="https://repository.apache.org">repository.apache.org</a> using your Apache ID.
Find your artifacts in the staging repository. Click on 'Staging Repositories' and look for a new one ending in "hbase" with a status of 'Open', select it.
Use the tree view to expand the list of repository contents and inspect if the artifacts you expect are present. Check the POMs.
As long as the staging repo is open you can re-upload if something is missing or built incorrectly.</p>
</div>
<div class="paragraph">
<p>If something is seriously wrong and you would like to back out the upload, you can use the 'Drop' button to drop and delete the staging repository.
Sometimes the upload fails in the middle. This is another reason you might have to 'Drop' the upload from the staging repository.</p>
</div>
<div class="paragraph">
<p>If it checks out, close the repo using the 'Close' button. The repository must be closed before a public URL to it becomes available. It may take a few minutes for the repository to close. Once complete you&#8217;ll see a public URL to the repository in the Nexus UI. You may also receive an email with the URL. Provide the URL to the temporary staging repository in the email that announces the release candidate.
(Folks will need to add this repo URL to their local poms or to their local <em>settings.xml</em> file to pull the published release candidate artifacts.)</p>
</div>
<div class="paragraph">
<p>When the release vote concludes successfully, return here and click the 'Release' button to release the artifacts to central. The release process will automatically drop and delete the staging repository.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">hbase-downstreamer</div>
<div class="paragraph">
<p>See the <a href="https://github.com/saintstack/hbase-downstreamer">hbase-downstreamer</a> test for a simple example of a project that is downstream of HBase an depends on it.
Check it out and run its simple test to make sure maven artifacts are properly deployed to the maven repository.
Be sure to edit the pom to point to the proper staging repository.
Make sure you are pulling from the repository when tests run and that you are not getting from your local repository, by either passing the <code>-U</code> flag or deleting your local repo content and check maven is pulling from remote out of the staging repository.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>See <a href="https://www.apache.org/dev/publishing-maven-artifacts.html">Publishing Maven Artifacts</a> for some pointers on this maven staging process.</p>
</div>
<div class="paragraph">
<p>If the HBase version ends in <code>-SNAPSHOT</code>, the artifacts go elsewhere.
They are put into the Apache snapshots repository directly and are immediately available.
Making a SNAPSHOT release, this is what you want to happen.</p>
</div>
<div class="paragraph">
<p>At this stage, you have two tarballs in your 'build output directory' and a set of artifacts
in a staging area of the maven repository, in the 'closed' state.
Next sign, fingerprint and then 'stage' your release candiate build output directory via svnpubsub by committing
your directory to <a href="https://dist.apache.org/repos/dist/dev/hbase/">The dev distribution directory</a>
(See comments on <a href="https://issues.apache.org/jira/browse/HBASE-10554">HBASE-10554 Please delete old releases from mirroring system</a>
but in essence it is an svn checkout of <a href="https://dist.apache.org/repos/dist/dev/hbase">dev/hbase</a>&#8201;&#8212;&#8201;releases are at
<a href="https://dist.apache.org/repos/dist/release/hbase">release/hbase</a>). In the <em>version directory</em> run the following commands:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ for i in *.tar.gz; do echo $i; gpg --print-md MD5 $i &gt; $i.md5 ; done
$ for i in *.tar.gz; do echo $i; gpg --print-md SHA512 $i &gt; $i.sha ; done
$ for i in *.tar.gz; do echo $i; gpg --armor --output $i.asc --detach-sig $i ; done
$ cd ..
# Presuming our 'build output directory' is named 0.96.0RC0, copy it to the svn checkout of the dist dev dir
# in this case named hbase.dist.dev.svn
$ cd /Users/stack/checkouts/hbase.dist.dev.svn
$ svn info
Path: .
Working Copy Root Path: /Users/stack/checkouts/hbase.dist.dev.svn
URL: https://dist.apache.org/repos/dist/dev/hbase
Repository Root: https://dist.apache.org/repos/dist
Repository UUID: 0d268c88-bc11-4956-87df-91683dc98e59
Revision: 15087
Node Kind: directory
Schedule: normal
Last Changed Author: ndimiduk
Last Changed Rev: 15045
Last Changed Date: 2016-08-28 11:13:36 -0700 (Sun, 28 Aug 2016)
$ mv 0.96.0RC0 /Users/stack/checkouts/hbase.dist.dev.svn
$ svn add 0.96.0RC0
$ svn commit ...</code></pre>
</div>
</div>
<div class="paragraph">
<p>Ensure it actually gets published by checking <a href="https://dist.apache.org/repos/dist/dev/hbase/">https://dist.apache.org/repos/dist/dev/hbase/</a>.</p>
</div>
<div class="paragraph">
<p>Announce the release candidate on the mailing list and call a vote.</p>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="maven.snapshot"><a class="anchor" href="#maven.snapshot"></a>177.2. Publishing a SNAPSHOT to maven</h3>
<div class="paragraph">
<p>Make sure your <em>settings.xml</em> is set up properly (see <a href="#maven.settings.xml">Example <em>~/.m2/settings.xml</em> File</a>).
Make sure the hbase version includes <code>-SNAPSHOT</code> as a suffix.
Following is an example of publishing SNAPSHOTS of a release that had an hbase version of 0.96.0 in its poms.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> $ mvn clean install -DskipTests javadoc:aggregate site assembly:single -Prelease
$ mvn -DskipTests deploy -Papache-release</code></pre>
</div>
</div>
<div class="paragraph">
<p>The <em>make_rc.sh</em> script mentioned above (see <a href="#maven.release">maven.release</a>) can help you publish <code>SNAPSHOTS</code>.
Make sure your <code>hbase.version</code> has a <code>-SNAPSHOT</code> suffix before running the script.
It will put a snapshot up into the apache snapshot repository for you.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.rc.voting"><a class="anchor" href="#hbase.rc.voting"></a>178. Voting on Release Candidates</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Everyone is encouraged to try and vote on HBase release candidates.
Only the votes of PMC members are binding.
PMC members, please read this WIP doc on policy voting for a release candidate, <a href="https://github.com/rectang/asfrelease/blob/master/release.md">Release Policy</a>.</p>
</div>
<div class="quoteblock">
<blockquote>
<div class="paragraph">
<p>Before casting +1 binding votes, individuals are required to
download the signed source code package onto their own hardware, compile it as
provided, and test the resulting executable on their own platform, along with also
validating cryptographic signatures and verifying that the package meets the
requirements of the ASF policy on releases.</p>
</div>
</blockquote>
</div>
<div class="paragraph">
<p>Regards the latter, run <code>mvn apache-rat:check</code> to verify all files are suitably licensed.
See <a href="http://search-hadoop.com/m/DHED4dhFaU">HBase, mail # dev - On recent discussion clarifying ASF release policy</a>
for how we arrived at this process.</p>
</div>
<div class="paragraph">
<p>To help with the release verification, please follow the guideline below and vote based on the your verification.</p>
</div>
<div class="sect2">
<h3 id="_baseline_verifications_for_voting_release_candidates"><a class="anchor" href="#_baseline_verifications_for_voting_release_candidates"></a>178.1. Baseline Verifications for Voting Release Candidates</h3>
<div class="paragraph">
<p>Although contributors have their own checklist for verifications, the following items are usually used for voting on release candidates.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>CHANGES.md if any</p>
</li>
<li>
<p>RELEASENOTES.md (release notes) if any</p>
</li>
<li>
<p>Generated API compatibility report</p>
<div class="ulist">
<ul>
<li>
<p>For what should be compatible please refer the <a href="https://hbase.apache.org/book.html#hbase.versioning">versioning guideline</a>, especially for items with marked as high severity</p>
</li>
</ul>
</div>
</li>
<li>
<p>Use <code>hbase-vote.sh</code> to perform sanity checks for checksum, signatures, files are licensed, built from source, and unit tests.</p>
<div class="ulist">
<ul>
<li>
<p><code>hbase-vote.sh</code> shell script is available under <code>dev-support</code> directory of HBase source. Following are the usage details.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">./dev-support/hbase-vote.sh -h
hbase-vote. A script for standard vote which verifies the following items
1. Checksum of sources and binaries
2. Signature of sources and binaries
3. Rat check
4. Built from source
5. Unit tests
Usage: hbase-vote.sh -s | --source &lt;url&gt; [-k | --key &lt;signature&gt;] [-f | --keys-file-url &lt;url&gt;] [-o | --output-dir &lt;/path/to/use&gt;]
hbase-vote.sh -h | --help
-h | --help Show this screen.
-s | --source '&lt;url&gt;' A URL pointing to the release candidate sources and binaries
e.g. https://dist.apache.org/repos/dist/dev/hbase/hbase-&lt;version&gt;RC0/
-k | --key '&lt;signature&gt;' A signature of the public key, e.g. 9AD2AE49
-f | --keys-file-url '&lt;url&gt;' the URL of the key file, default is
http://www.apache.org/dist/hbase/KEYS
-o | --output-dir '&lt;/path&gt;' directory which has the stdout and stderr of each verification target</code></pre>
</div>
</div>
<div class="ulist">
<ul>
<li>
<p>If you see any unit test failures, please call out the solo test result and whether it&#8217;s part of flaky (nightly) tests dashboard, e.g. <a href="https://builds.apache.org/view/H-L/view/HBase/job/HBase-Find-Flaky-Tests/job/master/lastSuccessfulBuild/artifact/dashboard.html">dashboard of master branch</a> (please change the test branch accordingly).</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_additional_verifications_for_voting_release_candidates"><a class="anchor" href="#_additional_verifications_for_voting_release_candidates"></a>178.2. Additional Verifications for Voting Release Candidates</h3>
<div class="paragraph">
<p>Other than the common verifications, contributors may call out additional concerns, e.g. for a specific feature by running end to end tests on a distributed environment. This is optional and always encouraged.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Start a distributed HBase cluster and call out the test result of specific workload on cluster. e.g.</p>
<div class="ulist">
<ul>
<li>
<p>Run basic table operations, e.g. <code>create/put/get/scan/flush/list/disable/drop</code></p>
</li>
<li>
<p>Run built-in tests, e.g. <code>LoadTestTool</code> (LTT) and <code>IntegrationTestBigLinkedList</code> (ITBLL)</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.release.announcement"><a class="anchor" href="#hbase.release.announcement"></a>179. Announcing Releases</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Once an RC has passed successfully and the needed artifacts have been staged for disribution, you&#8217;ll need to let everyone know about our shiny new release. It&#8217;s not a requirement, but to make things easier for release managers we have a template you can start with. Be sure you replace _version_ and other markers with the relevant version numbers. You should manually verify all links before sending.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="email">The HBase team is happy to announce the immediate availability of HBase _version_.
Apache HBase™ is an open-source, distributed, versioned, non-relational database.
Apache HBase gives you low latency random access to billions of rows with
millions of columns atop non-specialized hardware. To learn more about HBase,
see https://hbase.apache.org/.
HBase _version_ is the _nth_ minor release in the HBase _major_.x line, which aims to
improve the stability and reliability of HBase. This release includes roughly
XXX resolved issues not covered by previous _major_.x releases.
Notable new features include:
- List text descriptions of features that fit on one line
- Including if JDK or Hadoop support versions changes
- If the &quot;stable&quot; pointer changes, call that out
- For those with obvious JIRA IDs, include them (HBASE-YYYYY)
The full list of issues can be found in the included CHANGES.md and RELEASENOTES.md,
or via our issue tracker:
https://s.apache.org/hbase-_version_-jira
To download please follow the links and instructions on our website:
https://hbase.apache.org/downloads.html
Question, comments, and problems are always welcome at: dev@hbase.apache.org.
Thanks to all who contributed and made this release possible.
Cheers,
The HBase Dev Team</code></pre>
</div>
</div>
<div class="paragraph">
<p>You should sent this message to the following lists: <a href="mailto:dev@hbase.apache.org">dev@hbase.apache.org</a>, <a href="mailto:user@hbase.apache.org">user@hbase.apache.org</a>, <a href="mailto:announce@apache.org">announce@apache.org</a>. If you&#8217;d like a spot check before sending, feel free to ask via jira or the dev list.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="documentation"><a class="anchor" href="#documentation"></a>180. Generating the HBase Reference Guide</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The manual is marked up using Asciidoc.
We then use the <a href="http://asciidoctor.org/docs/asciidoctor-maven-plugin/">Asciidoctor maven plugin</a> to transform the markup to html.
This plugin is run when you specify the site goal as in when you run mvn site.
See <a href="#appendix_contributing_to_documentation">appendix contributing to documentation</a> for more information on building the documentation.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.org"><a class="anchor" href="#hbase.org"></a>181. Updating <a href="https://hbase.apache.org">hbase.apache.org</a></h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="hbase.org.site.contributing"><a class="anchor" href="#hbase.org.site.contributing"></a>181.1. Contributing to hbase.apache.org</h3>
<div class="paragraph">
<p>See <a href="#appendix_contributing_to_documentation">appendix contributing to documentation</a> for more information on contributing to the documentation or website.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbase.org.site.publishing"><a class="anchor" href="#hbase.org.site.publishing"></a>181.2. Publishing <a href="https://hbase.apache.org">hbase.apache.org</a></h3>
<div class="paragraph">
<p>See <a href="#website_publish">Publishing the HBase Website and Documentation</a> for instructions on publishing the website and documentation.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.tests"><a class="anchor" href="#hbase.tests"></a>182. Tests</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Developers, at a minimum, should familiarize themselves with the unit test detail; unit tests in HBase have a character not usually seen in other projects.</p>
</div>
<div class="paragraph">
<p>This information is about unit tests for HBase itself.
For developing unit tests for your HBase applications, see <a href="#unit.tests">unit.tests</a>.</p>
</div>
<div class="sect2">
<h3 id="hbase.moduletests"><a class="anchor" href="#hbase.moduletests"></a>182.1. Apache HBase Modules</h3>
<div class="paragraph">
<p>As of 0.96, Apache HBase is split into multiple modules.
This creates "interesting" rules for how and where tests are written.
If you are writing code for <code>hbase-server</code>, see <a href="#hbase.unittests">hbase.unittests</a> for how to write your tests.
These tests can spin up a minicluster and will need to be categorized.
For any other module, for example <code>hbase-common</code>, the tests must be strict unit tests and just test the class under test - no use of the HBaseTestingUtility or minicluster is allowed (or even possible given the dependency tree).</p>
</div>
<div class="sect3">
<h4 id="hbase.moduletest.shell"><a class="anchor" href="#hbase.moduletest.shell"></a>182.1.1. Testing the HBase Shell</h4>
<div class="paragraph">
<p>The HBase shell and its tests are predominantly written in jruby.</p>
</div>
<div class="paragraph">
<p>In order to make these tests run as a part of the standard build, there are a few JUnit test classes that take care of loading the jruby implemented tests and running them.
The tests were split into separate classes to accomodate class level timeouts (see <a href="#hbase.unittests">Unit Tests</a> for specifics).
You can run all of these tests from the top level with:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> mvn clean test -Dtest=Test*Shell</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you have previously done a <code>mvn install</code>, then you can instruct maven to run only the tests in the hbase-shell module with:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> mvn clean test -pl hbase-shell</code></pre>
</div>
</div>
<div class="paragraph">
<p>Alternatively, you may limit the shell tests that run using the system variable <code>shell.test</code>.
This value should specify the ruby literal equivalent of a particular test case by name.
For example, the tests that cover the shell commands for altering tables are contained in the test case <code>AdminAlterTableTest</code> and you can run them with:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> mvn clean test -pl hbase-shell -Dshell.test=/AdminAlterTableTest/</code></pre>
</div>
</div>
<div class="paragraph">
<p>You may also use a <a href="http://docs.ruby-doc.com/docs/ProgrammingRuby/html/language.html#UJ">Ruby Regular Expression
literal</a> (in the <code>/pattern/</code> style) to select a set of test cases.
You can run all of the HBase admin related tests, including both the normal administration and the security administration, with the command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> mvn clean test -pl hbase-shell -Dshell.test=/.*Admin.*Test/</code></pre>
</div>
</div>
<div class="paragraph">
<p>In the event of a test failure, you can see details by examining the XML version of the surefire report results</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne"> vim hbase-shell/target/surefire-reports/TEST-org.apache.hadoop.hbase.client.TestShell.xml</code></pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="hbase.moduletest.run"><a class="anchor" href="#hbase.moduletest.run"></a>182.1.2. Running Tests in other Modules</h4>
<div class="paragraph">
<p>If the module you are developing in has no other dependencies on other HBase modules, then you can cd into that module and just run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn test</code></pre>
</div>
</div>
<div class="paragraph">
<p>which will just run the tests IN THAT MODULE.
If there are other dependencies on other modules, then you will have run the command from the ROOT HBASE DIRECTORY.
This will run the tests in the other modules, unless you specify to skip the tests in that module.
For instance, to skip the tests in the hbase-server module, you would run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean test -PskipServerTests</code></pre>
</div>
</div>
<div class="paragraph">
<p>from the top level directory to run all the tests in modules other than hbase-server.
Note that you can specify to skip tests in multiple modules as well as just for a single module.
For example, to skip the tests in <code>hbase-server</code> and <code>hbase-common</code>, you would run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean test -PskipServerTests -PskipCommonTests</code></pre>
</div>
</div>
<div class="paragraph">
<p>Also, keep in mind that if you are running tests in the <code>hbase-server</code> module you will need to apply the maven profiles discussed in <a href="#hbase.unittests.cmds">hbase.unittests.cmds</a> to get the tests to run properly.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.unittests"><a class="anchor" href="#hbase.unittests"></a>182.2. Unit Tests</h3>
<div class="paragraph">
<p>Apache HBase unit tests must carry a Category annotation and
as of <code>hbase-2.0.0</code>, must be stamped with the HBase <code>ClassRule</code>.
Here is an example of what a Test Class looks like with a
Category and ClassRule included:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
<span class="annotation">@Category</span>(SmallTests.class)
<span class="directive">public</span> <span class="type">class</span> <span class="class">TestHRegionInfo</span> {
<span class="annotation">@ClassRule</span>
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestHRegionInfo.class);
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testCreateHRegionInfoName() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="comment">// ...</span>
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Here the Test Class is <code>TestHRegionInfo</code>. The <code>CLASS_RULE</code> has
the same form in every test class only the <code>.class</code> you pass
is that of the local test; i.e. in the TestTimeout Test Class, you&#8217;d
pass <code>TestTimeout.class</code> to the <code>CLASS_RULE</code> instead of the
<code>TestHRegionInfo.class</code> we have above. The <code>CLASS_RULE</code>
is where we&#8217;ll enforce timeouts (currently set at a hard-limit of
thirteen! minutes for all tests&#8201;&#8212;&#8201;780 seconds) and other cross-unit test facility.
The test is in the <code>SmallTest</code> Category.</p>
</div>
<div class="paragraph">
<p>Categories can be arbitrary and provided as a list but each test MUST
carry one from the following list of sizings: <code>small</code>, <code>medium</code>, <code>large</code>, and
<code>integration</code>. The test sizing is designated using the JUnit
<a href="https://github.com/junit-team/junit4/wiki/Categories">categories</a>: <code>SmallTests</code>, <code>MediumTests</code>, <code>LargeTests</code>, <code>IntegrationTests</code>.
JUnit Categories are denoted using java annotations (a special unit test looks
for the presence of the @Category annotation in all unit tess and will fail if it
finds a test suite missing a sizing marking).</p>
</div>
<div class="paragraph">
<p>The first three categories, <code>small</code>, <code>medium</code>, and <code>large</code>, are for test cases which run when you
type <code>$ mvn test</code>.
In other words, these three categorizations are for HBase unit tests.
The <code>integration</code> category is not for unit tests, but for integration tests.
These are normally run when you invoke <code>$ mvn verify</code>.
Integration tests are described in <a href="#integration.tests">integration.tests</a>.</p>
</div>
<div class="paragraph">
<p>Keep reading to figure which annotation of the set <code>small</code>, <code>medium</code>, and <code>large</code>
to put on your new HBase test case.</p>
</div>
<div class="dlist">
<div class="title">Categorizing Tests</div>
<dl>
<dt class="hdlist1">Small Tests </dt>
<dd>
<p><em>Small</em> test cases are executed in a shared JVM and each test suite/test class should
run in 15 seconds or less; i.e. a <a href="https://en.wikipedia.org/wiki/JUnit">junit test fixture</a>, a java object made
up of test methods, should finish in under 15 seconds, no matter how many or how few test methods
it has. These test cases should not use a minicluster.</p>
</dd>
<dt class="hdlist1">Medium Tests </dt>
<dd>
<p><em>Medium</em> test cases are executed in separate JVM and individual test suites or test classes or in
junit parlance, <a href="https://en.wikipedia.org/wiki/JUnit">test fixture</a>, should run in 50 seconds
or less. These test cases can use a mini cluster.</p>
</dd>
<dt class="hdlist1">Large Tests </dt>
<dd>
<p><em>Large</em> test cases are everything else. They are typically large-scale tests, regression tests
for specific bugs, timeout tests, or performance tests. No large test suite can take longer than
ten minutes. It will be killed as timed out. Cast your test as an Integration Test if it needs
to run longer.</p>
</dd>
<dt class="hdlist1">Integration Tests </dt>
<dd>
<p><em>Integration</em> tests are system level tests.
See <a href="#integration.tests">integration.tests</a> for more info.
If you invoke <code>$ mvn test</code> on integration tests, there is no timeout for the test.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="hbase.unittests.cmds"><a class="anchor" href="#hbase.unittests.cmds"></a>182.3. Running tests</h3>
<div class="sect3">
<h4 id="hbase.unittests.cmds.test"><a class="anchor" href="#hbase.unittests.cmds.test"></a>182.3.1. Default: small and medium category tests</h4>
<div class="paragraph">
<p>Running <code>mvn test</code> will execute all small tests in a single JVM (no fork) and then medium tests in a separate JVM for each test instance.
Medium tests are NOT executed if there is an error in a small test. Large tests are NOT executed.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.cmds.test.runalltests"><a class="anchor" href="#hbase.unittests.cmds.test.runalltests"></a>182.3.2. Running all tests</h4>
<div class="paragraph">
<p>Running <code>mvn test -P runAllTests</code> will execute small tests in a single JVM then medium and large tests in a separate JVM for each test.
Medium and large tests are NOT executed if there is an error in a small test.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.cmds.test.localtests.mytest"><a class="anchor" href="#hbase.unittests.cmds.test.localtests.mytest"></a>182.3.3. Running a single test or all tests in a package</h4>
<div class="paragraph">
<p>To run an individual test, e.g. <code>MyTest</code>, rum <code>mvn test -Dtest=MyTest</code> You can also pass multiple, individual tests as a comma-delimited list:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">mvn test -Dtest=MyTest1,MyTest2,MyTest3</code></pre>
</div>
</div>
<div class="paragraph">
<p>You can also pass a package, which will run all tests under the package:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">mvn test '-Dtest=org.apache.hadoop.hbase.client.*'</code></pre>
</div>
</div>
<div class="paragraph">
<p>When <code>-Dtest</code> is specified, the <code>localTests</code> profile will be used.
Each junit test is executed in a separate JVM (A fork per test class). There is no parallelization when tests are running in this mode.
You will see a new message at the end of the -report: <code>"[INFO] Tests are skipped"</code>.
It&#8217;s harmless.
However, you need to make sure the sum of <code>Tests run:</code> in the <code>Results:</code> section of test reports matching the number of tests you specified because no error will be reported when a non-existent test case is specified.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.cmds.test.profiles"><a class="anchor" href="#hbase.unittests.cmds.test.profiles"></a>182.3.4. Other test invocation permutations</h4>
<div class="paragraph">
<p>Running <code>mvn test -P runSmallTests</code> will execute "small" tests only, using a single JVM.</p>
</div>
<div class="paragraph">
<p>Running <code>mvn test -P runMediumTests</code> will execute "medium" tests only, launching a new JVM for each test-class.</p>
</div>
<div class="paragraph">
<p>Running <code>mvn test -P runLargeTests</code> will execute "large" tests only, launching a new JVM for each test-class.</p>
</div>
<div class="paragraph">
<p>For convenience, you can run <code>mvn test -P runDevTests</code> to execute both small and medium tests, using a single JVM.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.test.faster"><a class="anchor" href="#hbase.unittests.test.faster"></a>182.3.5. Running tests faster</h4>
<div class="paragraph">
<p>By default, <code>$ mvn test -P runAllTests</code> runs 5 tests in parallel.
It can be increased on a developer&#8217;s machine.
Allowing that you can have 2 tests in parallel per core, and you need about 2GB of memory per test (at the extreme), if you have an 8 core, 24GB box, you can have 16 tests in parallel.
but the memory available limits it to 12 (24/2), To run all tests with 12 tests in parallel, do this: mvn test -P runAllTests
-Dsurefire.secondPartForkCount=12.
If using a version earlier than 2.0, do: +mvn test -P runAllTests -Dsurefire.secondPartThreadCount=12
+.
To increase the speed, you can as well use a ramdisk.
You will need 2GB of memory to run all tests.
You will also need to delete the files between two test run.
The typical way to configure a ramdisk on Linux is:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ sudo mkdir /ram2G
sudo mount -t tmpfs -o size=2048M tmpfs /ram2G</pre>
</div>
</div>
<div class="paragraph">
<p>You can then use it to run all HBase tests on 2.0 with the command:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn test
-P runAllTests -Dsurefire.secondPartForkCount=12
-Dtest.build.data.basedirectory=/ram2G</pre>
</div>
</div>
<div class="paragraph">
<p>On earlier versions, use:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn test
-P runAllTests -Dsurefire.secondPartThreadCount=12
-Dtest.build.data.basedirectory=/ram2G</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.cmds.test.hbasetests"><a class="anchor" href="#hbase.unittests.cmds.test.hbasetests"></a>182.3.6. hbasetests.sh</h4>
<div class="paragraph">
<p>It&#8217;s also possible to use the script hbasetests.sh.
This script runs the medium and large tests in parallel with two maven instances, and provides a single report.
This script does not use the hbase version of surefire so no parallelization is being done other than the two maven instances the script sets up.
It must be executed from the directory which contains the <em>pom.xml</em>.</p>
</div>
<div class="paragraph">
<p>For example running ./dev-support/hbasetests.sh will execute small and medium tests.
Running ./dev-support/hbasetests.sh
runAllTests will execute all tests.
Running ./dev-support/hbasetests.sh replayFailed will rerun the failed tests a second time, in a separate jvm and without parallelisation.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.timeouts"><a class="anchor" href="#hbase.unittests.timeouts"></a>182.3.7. Test Timeouts</h4>
<div class="paragraph">
<p>The HBase unit test sizing Categorization timeouts are not strictly enforced.</p>
</div>
<div class="paragraph">
<p>Any test that runs longer than ten minutes will be timedout/killed.</p>
</div>
<div class="paragraph">
<p>As of hbase-2.0.0, we have purged all per-test-method timeouts: i.e.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
<span class="annotation">@Test</span>(timeout=<span class="integer">30000</span>)
<span class="directive">public</span> <span class="type">void</span> testCreateHRegionInfoName() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="comment">// ...</span>
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>They are discouraged and don&#8217;t make much sense given we are timing
base of how long the whole Test Fixture/Class/Suite takes and
that the variance in how long a test method takes varies wildly
dependent upon context (loaded Apache Infrastructure versus
developer machine with nothing else running on it).</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.unittests.resource.checker"><a class="anchor" href="#hbase.unittests.resource.checker"></a>182.3.8. Test Resource Checker</h4>
<div class="paragraph">
<p>A custom Maven SureFire plugin listener checks a number of resources before and after each HBase unit test runs and logs its findings at the end of the test output files which can be found in <em>target/surefire-reports</em> per Maven module (Tests write test reports named for the test class into this directory.
Check the <em>*-out.txt</em> files). The resources counted are the number of threads, the number of file descriptors, etc.
If the number has increased, it adds a <em>LEAK?</em> comment in the logs.
As you can have an HBase instance running in the background, some threads can be deleted/created without any specific action in the test.
However, if the test does not work as expected, or if the test should not impact these resources, it&#8217;s worth checking these log lines <span class="computeroutput">...hbase.ResourceChecker(157): before...</span> and <span class="computeroutput">...hbase.ResourceChecker(157): after...</span>.
For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>2012-09-26 09:22:15,315 INFO [pool-1-thread-1]
hbase.ResourceChecker(157): after:
regionserver.TestColumnSeeking#testReseeking Thread=65 (was 65),
OpenFileDescriptor=107 (was 107), MaxFileDescriptor=10240 (was 10240),
ConnectionCount=1 (was 1)</pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hbase.tests.writing"><a class="anchor" href="#hbase.tests.writing"></a>182.4. Writing Tests</h3>
<div class="sect3">
<h4 id="hbase.tests.rules"><a class="anchor" href="#hbase.tests.rules"></a>182.4.1. General rules</h4>
<div class="ulist">
<ul>
<li>
<p>As much as possible, tests should be written as category small tests.</p>
</li>
<li>
<p>All tests must be written to support parallel execution on the same machine, hence they should not use shared resources as fixed ports or fixed file names.</p>
</li>
<li>
<p>Tests should not overlog.
More than 100 lines/second makes the logs complex to read and use i/o that are hence not available for the other tests.</p>
</li>
<li>
<p>Tests can be written with <code>HBaseTestingUtility</code>.
This class offers helper functions to create a temp directory and do the cleanup, or to start a cluster.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="hbase.tests.categories"><a class="anchor" href="#hbase.tests.categories"></a>182.4.2. Categories and execution time</h4>
<div class="ulist">
<ul>
<li>
<p>All tests must be categorized, if not they could be skipped.</p>
</li>
<li>
<p>All tests should be written to be as fast as possible.</p>
</li>
<li>
<p>See <a href="#hbase.unittests">hbase.unittests</a> for test case categories and corresponding timeouts.
This should ensure a good parallelization for people using it, and ease the analysis when the test fails.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="hbase.tests.sleeps"><a class="anchor" href="#hbase.tests.sleeps"></a>182.4.3. Sleeps in tests</h4>
<div class="paragraph">
<p>Whenever possible, tests should not use <span class="method">Thread.sleep</span>, but rather waiting for the real event they need.
This is faster and clearer for the reader.
Tests should not do a <span class="method">Thread.sleep</span> without testing an ending condition.
This allows understanding what the test is waiting for.
Moreover, the test will work whatever the machine performance is.
Sleep should be minimal to be as fast as possible.
Waiting for a variable should be done in a 40ms sleep loop.
Waiting for a socket operation should be done in a 200 ms sleep loop.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.tests.cluster"><a class="anchor" href="#hbase.tests.cluster"></a>182.4.4. Tests using a cluster</h4>
<div class="paragraph">
<p>Tests using a HRegion do not have to start a cluster: A region can use the local file system.
Start/stopping a cluster cost around 10 seconds.
They should not be started per test method but per test class.
Started cluster must be shutdown using <span class="method">HBaseTestingUtility#shutdownMiniCluster</span>, which cleans the directories.
As most as possible, tests should use the default settings for the cluster.
When they don&#8217;t, they should document it.
This will allow to share the cluster later.</p>
</div>
</div>
<div class="sect3">
<h4 id="hbase.tests.example.code"><a class="anchor" href="#hbase.tests.example.code"></a>182.4.5. Tests Skeleton Code</h4>
<div class="paragraph">
<p>Here is a test skeleton code with Categorization and a Category-based timeout rule to copy and paste and use as basis for test contribution.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="comment">/**
* Describe what this testcase tests. Talk about resources initialized in @BeforeClass (before
* any test is run) and before each test is run, etc.
*/</span>
<span class="comment">// Specify the category as explained in &lt;&lt;hbase.unittests,hbase.unittests&gt;&gt;.</span>
<span class="annotation">@Category</span>(SmallTests.class)
<span class="directive">public</span> <span class="type">class</span> <span class="class">TestExample</span> {
<span class="comment">// Replace the TestExample.class in the below with the name of your test fixture class.</span>
<span class="directive">private</span> <span class="directive">static</span> <span class="directive">final</span> Log LOG = LogFactory.getLog(TestExample.class);
<span class="comment">// Handy test rule that allows you subsequently get the name of the current method. See</span>
<span class="comment">// down in 'testExampleFoo()' where we use it to log current test's name.</span>
<span class="annotation">@Rule</span> <span class="directive">public</span> TestName testName = <span class="keyword">new</span> TestName();
<span class="comment">// The below rule does two things. It decides the timeout based on the category</span>
<span class="comment">// (small/medium/large) of the testcase. This @Rule requires that the full testcase runs</span>
<span class="comment">// within this timeout irrespective of individual test methods' times. The second</span>
<span class="comment">// feature is we'll dump in the log when the test is done a count of threads still</span>
<span class="comment">// running.</span>
<span class="annotation">@Rule</span> <span class="directive">public</span> <span class="directive">static</span> TestRule timeout = CategoryBasedTimeout.builder().
withTimeout(<span class="local-variable">this</span>.getClass()).withLookingForStuckThread(<span class="predefined-constant">true</span>).build();
<span class="annotation">@Before</span>
<span class="directive">public</span> <span class="type">void</span> setUp() <span class="directive">throws</span> <span class="exception">Exception</span> {
}
<span class="annotation">@After</span>
<span class="directive">public</span> <span class="type">void</span> tearDown() <span class="directive">throws</span> <span class="exception">Exception</span> {
}
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testExampleFoo() {
LOG.info(<span class="string"><span class="delimiter">&quot;</span><span class="content">Running test </span><span class="delimiter">&quot;</span></span> + testName.getMethodName());
}
}</code></pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="integration.tests"><a class="anchor" href="#integration.tests"></a>182.5. Integration Tests</h3>
<div class="paragraph">
<p>HBase integration/system tests are tests that are beyond HBase unit tests.
They are generally long-lasting, sizeable (the test can be asked to 1M rows or 1B rows), targetable (they can take configuration that will point them at the ready-made cluster they are to run against; integration tests do not include cluster start/stop code), and verifying success, integration tests rely on public APIs only; they do not attempt to examine server internals asserting success/fail.
Integration tests are what you would run when you need to more elaborate proofing of a release candidate beyond what unit tests can do.
They are not generally run on the Apache Continuous Integration build server, however, some sites opt to run integration tests as a part of their continuous testing on an actual cluster.</p>
</div>
<div class="paragraph">
<p>Integration tests currently live under the <em>src/test</em> directory in the hbase-it submodule and will match the regex: <em>*<strong>/IntegrationTest</strong>.java</em>.
All integration tests are also annotated with <code>@Category(IntegrationTests.class)</code>.</p>
</div>
<div class="paragraph">
<p>Integration tests can be run in two modes: using a mini cluster, or against an actual distributed cluster.
Maven failsafe is used to run the tests using the mini cluster.
IntegrationTestsDriver class is used for executing the tests against a distributed cluster.
Integration tests SHOULD NOT assume that they are running against a mini cluster, and SHOULD NOT use private API&#8217;s to access cluster state.
To interact with the distributed or mini cluster uniformly, <code>IntegrationTestingUtility</code>, and <code>HBaseCluster</code> classes, and public client API&#8217;s can be used.</p>
</div>
<div class="paragraph">
<p>On a distributed cluster, integration tests that use ChaosMonkey or otherwise manipulate services thru cluster manager (e.g.
restart regionservers) use SSH to do it.
To run these, test process should be able to run commands on remote end, so ssh should be configured accordingly (for example, if HBase runs under hbase user in your cluster, you can set up passwordless ssh for that user and run the test also under it). To facilitate that, <code>hbase.it.clustermanager.ssh.user</code>, <code>hbase.it.clustermanager.ssh.opts</code> and <code>hbase.it.clustermanager.ssh.cmd</code> configuration settings can be used.
"User" is the remote user that cluster manager should use to perform ssh commands.
"Opts" contains additional options that are passed to SSH (for example, "-i /tmp/my-key"). Finally, if you have some custom environment setup, "cmd" is the override format for the entire tunnel (ssh) command.
The default string is {<code>/usr/bin/ssh %1$s %2$s%3$s%4$s "%5$s"</code>} and is a good starting point.
This is a standard Java format string with 5 arguments that is used to execute the remote command.
The argument 1 (%1$s) is SSH options set the via opts setting or via environment variable, 2 is SSH user name, 3 is "@" if username is set or "" otherwise, 4 is the target host name, and 5 is the logical command to execute (that may include single quotes, so don&#8217;t use them). For example, if you run the tests under non-hbase user and want to ssh as that user and change to hbase on remote machine, you can use:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">/usr/bin/ssh %1$s %2$s%3$s%4$s &quot;su hbase - -c \&quot;%5$s\&quot;&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>That way, to kill RS (for example) integration tests may run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">{/usr/bin/ssh some-hostname &quot;su hbase - -c \&quot;ps aux | ... | kill ...\&quot;&quot;}</code></pre>
</div>
</div>
<div class="paragraph">
<p>The command is logged in the test logs, so you can verify it is correct for your environment.</p>
</div>
<div class="paragraph">
<p>To disable the running of Integration Tests, pass the following profile on the command line <code>-PskipIntegrationTests</code>.
For example,</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> mvn clean install test -Dtest=TestZooKeeper -PskipIntegrationTests</code></pre>
</div>
</div>
<div class="sect3">
<h4 id="maven.build.commands.integration.tests.mini"><a class="anchor" href="#maven.build.commands.integration.tests.mini"></a>182.5.1. Running integration tests against mini cluster</h4>
<div class="paragraph">
<p>HBase 0.92 added a <code>verify</code> maven target.
Invoking it, for example by doing <code>mvn verify</code>, will run all the phases up to and including the verify phase via the maven <a href="https://maven.apache.org/plugins/maven-failsafe-plugin/">failsafe
plugin</a>, running all the above mentioned HBase unit tests as well as tests that are in the HBase integration test group.
After you have completed mvn install -DskipTests You can run just the integration tests by invoking:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">cd hbase-it
mvn verify</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you just want to run the integration tests in top-level, you need to run two commands.
First:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn failsafe:integration-test</pre>
</div>
</div>
<div class="paragraph">
<p>This actually runs ALL the integration tests.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
This command will always output <code>BUILD SUCCESS</code> even if there are test failures.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>At this point, you could grep the output by hand looking for failed tests.
However, maven will do this for us; just use:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn failsafe:verify</pre>
</div>
</div>
<div class="paragraph">
<p>The above command basically looks at all the test results (so don&#8217;t remove the 'target' directory) for test failures and reports the results.</p>
</div>
<div class="sect4">
<h5 id="maven.build.commands.integration.tests2"><a class="anchor" href="#maven.build.commands.integration.tests2"></a>Running a subset of Integration tests</h5>
<div class="paragraph">
<p>This is very similar to how you specify running a subset of unit tests (see above), but use the property <code>it.test</code> instead of <code>test</code>.
To just run <code>IntegrationTestClassXYZ.java</code>, use:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn failsafe:integration-test -Dit.test=IntegrationTestClassXYZ -DfailIfNoTests=false</pre>
</div>
</div>
<div class="paragraph">
<p>The next thing you might want to do is run groups of integration tests, say all integration tests that are named IntegrationTestClassX*.java:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn failsafe:integration-test -Dit.test=*ClassX* -DfailIfNoTests=false</pre>
</div>
</div>
<div class="paragraph">
<p>This runs everything that is an integration test that matches <strong>ClassX</strong>. This means anything matching: "<strong>*/IntegrationTest*ClassX</strong>". You can also run multiple groups of integration tests using comma-delimited lists (similar to unit tests). Using a list of matches still supports full regex matching for each of the groups. This would look something like:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>mvn failsafe:integration-test -Dit.test=*ClassX*,*ClassY -DfailIfNoTests=false</pre>
</div>
</div>
</div>
</div>
<div class="sect3">
<h4 id="maven.build.commands.integration.tests.distributed"><a class="anchor" href="#maven.build.commands.integration.tests.distributed"></a>182.5.2. Running integration tests against distributed cluster</h4>
<div class="paragraph">
<p>If you have an already-setup HBase cluster, you can launch the integration tests by invoking the class <code>IntegrationTestsDriver</code>.
You may have to run test-compile first.
The configuration will be picked by the bin/hbase script.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn test-compile</code></pre>
</div>
</div>
<div class="paragraph">
<p>Then launch the tests with:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">bin/hbase [--config config_dir] org.apache.hadoop.hbase.IntegrationTestsDriver</code></pre>
</div>
</div>
<div class="paragraph">
<p>Pass <code>-h</code> to get usage on this sweet tool.
Running the IntegrationTestsDriver without any argument will launch tests found under <code>hbase-it/src/test</code>, having <code>@Category(IntegrationTests.class)</code> annotation, and a name starting with <code>IntegrationTests</code>.
See the usage, by passing -h, to see how to filter test classes.
You can pass a regex which is checked against the full class name; so, part of class name can be used.
IntegrationTestsDriver uses Junit to run the tests.
Currently there is no support for running integration tests against a distributed cluster using maven (see <a href="https://issues.apache.org/jira/browse/HBASE-6201">HBASE-6201</a>).</p>
</div>
<div class="paragraph">
<p>The tests interact with the distributed cluster by using the methods in the <code>DistributedHBaseCluster</code> (implementing <code>HBaseCluster</code>) class, which in turn uses a pluggable <code>ClusterManager</code>.
Concrete implementations provide actual functionality for carrying out deployment-specific and environment-dependent tasks (SSH, etc). The default <code>ClusterManager</code> is <code>HBaseClusterManager</code>, which uses SSH to remotely execute start/stop/kill/signal commands, and assumes some posix commands (ps, etc). Also assumes the user running the test has enough "power" to start/stop servers on the remote machines.
By default, it picks up <code>HBASE_SSH_OPTS</code>, <code>HBASE_HOME</code>, <code>HBASE_CONF_DIR</code> from the env, and uses <code>bin/hbase-daemon.sh</code> to carry out the actions.
Currently tarball deployments, deployments which uses <em>hbase-daemons.sh</em>, and <a href="https://incubator.apache.org/ambari/">Apache Ambari</a> deployments are supported.
<em>/etc/init.d/</em> scripts are not supported for now, but it can be easily added.
For other deployment options, a ClusterManager can be implemented and plugged in.</p>
</div>
</div>
<div class="sect3">
<h4 id="maven.build.commands.integration.tests.destructive"><a class="anchor" href="#maven.build.commands.integration.tests.destructive"></a>182.5.3. Destructive integration / system tests (ChaosMonkey)</h4>
<div class="paragraph">
<p>HBase 0.96 introduced a tool named <code>ChaosMonkey</code>, modeled after
<a href="https://netflix.github.io/chaosmonkey/">same-named tool by Netflix&#8217;s Chaos Monkey tool</a>.
ChaosMonkey simulates real-world
faults in a running cluster by killing or disconnecting random servers, or injecting
other failures into the environment. You can use ChaosMonkey as a stand-alone tool
to run a policy while other tests are running. In some environments, ChaosMonkey is
always running, in order to constantly check that high availability and fault tolerance
are working as expected.</p>
</div>
<div class="paragraph">
<p>ChaosMonkey defines <strong>Actions</strong> and <strong>Policies</strong>.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Actions</dt>
<dd>
<p>Actions are predefined sequences of events, such as the following:</p>
<div class="ulist">
<ul>
<li>
<p>Restart active master (sleep 5 sec)</p>
</li>
<li>
<p>Restart random regionserver (sleep 5 sec)</p>
</li>
<li>
<p>Restart random regionserver (sleep 60 sec)</p>
</li>
<li>
<p>Restart META regionserver (sleep 5 sec)</p>
</li>
<li>
<p>Restart ROOT regionserver (sleep 5 sec)</p>
</li>
<li>
<p>Batch restart of 50% of regionservers (sleep 5 sec)</p>
</li>
<li>
<p>Rolling restart of 100% of regionservers (sleep 5 sec)</p>
</li>
</ul>
</div>
</dd>
<dt class="hdlist1">Policies</dt>
<dd>
<p>A policy is a strategy for executing one or more actions. The default policy
executes a random action every minute based on predefined action weights.
A given policy will be executed until ChaosMonkey is interrupted.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>Most ChaosMonkey actions are configured to have reasonable defaults, so you can run
ChaosMonkey against an existing cluster without any additional configuration. The
following example runs ChaosMonkey with the default configuration:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bash">$ bin/hbase org.apache.hadoop.hbase.util.ChaosMonkey
12/11/19 23:21:57 INFO util.ChaosMonkey: Using ChaosMonkey Policy: class org.apache.hadoop.hbase.util.ChaosMonkey$PeriodicRandomActionPolicy, period:60000
12/11/19 23:21:57 INFO util.ChaosMonkey: Sleeping for 26953 to add jitter
12/11/19 23:22:24 INFO util.ChaosMonkey: Performing action: Restart active master
12/11/19 23:22:24 INFO util.ChaosMonkey: Killing master:master.example.com,60000,1353367210440
12/11/19 23:22:24 INFO hbase.HBaseCluster: Aborting Master: master.example.com,60000,1353367210440
12/11/19 23:22:24 INFO hbase.ClusterManager: Executing remote command: ps aux | grep master | grep -v grep | tr -s ' ' | cut -d ' ' -f2 | xargs kill -s SIGKILL , hostname:master.example.com
12/11/19 23:22:25 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:
12/11/19 23:22:25 INFO hbase.HBaseCluster: Waiting service:master to stop: master.example.com,60000,1353367210440
12/11/19 23:22:25 INFO hbase.ClusterManager: Executing remote command: ps aux | grep master | grep -v grep | tr -s ' ' | cut -d ' ' -f2 , hostname:master.example.com
12/11/19 23:22:25 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:
12/11/19 23:22:25 INFO util.ChaosMonkey: Killed master server:master.example.com,60000,1353367210440
12/11/19 23:22:25 INFO util.ChaosMonkey: Sleeping for:5000
12/11/19 23:22:30 INFO util.ChaosMonkey: Starting master:master.example.com
12/11/19 23:22:30 INFO hbase.HBaseCluster: Starting Master on: master.example.com
12/11/19 23:22:30 INFO hbase.ClusterManager: Executing remote command: /homes/enis/code/hbase-0.94/bin/../bin/hbase-daemon.sh --config /homes/enis/code/hbase-0.94/bin/../conf start master , hostname:master.example.com
12/11/19 23:22:31 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:starting master, logging to /homes/enis/code/hbase-0.94/bin/../logs/hbase-enis-master-master.example.com.out
....
12/11/19 23:22:33 INFO util.ChaosMonkey: Started master: master.example.com,60000,1353367210440
12/11/19 23:22:33 INFO util.ChaosMonkey: Sleeping for:51321
12/11/19 23:23:24 INFO util.ChaosMonkey: Performing action: Restart random region server
12/11/19 23:23:24 INFO util.ChaosMonkey: Killing region server:rs3.example.com,60020,1353367027826
12/11/19 23:23:24 INFO hbase.HBaseCluster: Aborting RS: rs3.example.com,60020,1353367027826
12/11/19 23:23:24 INFO hbase.ClusterManager: Executing remote command: ps aux | grep regionserver | grep -v grep | tr -s ' ' | cut -d ' ' -f2 | xargs kill -s SIGKILL , hostname:rs3.example.com
12/11/19 23:23:25 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:
12/11/19 23:23:25 INFO hbase.HBaseCluster: Waiting service:regionserver to stop: rs3.example.com,60020,1353367027826
12/11/19 23:23:25 INFO hbase.ClusterManager: Executing remote command: ps aux | grep regionserver | grep -v grep | tr -s ' ' | cut -d ' ' -f2 , hostname:rs3.example.com
12/11/19 23:23:25 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:
12/11/19 23:23:25 INFO util.ChaosMonkey: Killed region server:rs3.example.com,60020,1353367027826. Reported num of rs:6
12/11/19 23:23:25 INFO util.ChaosMonkey: Sleeping for:60000
12/11/19 23:24:25 INFO util.ChaosMonkey: Starting region server:rs3.example.com
12/11/19 23:24:25 INFO hbase.HBaseCluster: Starting RS on: rs3.example.com
12/11/19 23:24:25 INFO hbase.ClusterManager: Executing remote command: /homes/enis/code/hbase-0.94/bin/../bin/hbase-daemon.sh --config /homes/enis/code/hbase-0.94/bin/../conf start regionserver , hostname:rs3.example.com
12/11/19 23:24:26 INFO hbase.ClusterManager: Executed remote command, exit code:0 , output:starting regionserver, logging to /homes/enis/code/hbase-0.94/bin/../logs/hbase-enis-regionserver-rs3.example.com.out
12/11/19 23:24:27 INFO util.ChaosMonkey: Started region server:rs3.example.com,60020,1353367027826. Reported num of rs:6</code></pre>
</div>
</div>
<div class="paragraph">
<p>The output indicates that ChaosMonkey started the default <code>PeriodicRandomActionPolicy</code>
policy, which is configured with all the available actions. It chose to run <code>RestartActiveMaster</code> and <code>RestartRandomRs</code> actions.</p>
</div>
</div>
<div class="sect3">
<h4 id="_available_policies"><a class="anchor" href="#_available_policies"></a>182.5.4. Available Policies</h4>
<div class="paragraph">
<p>HBase ships with several ChaosMonkey policies, available in the
<code>hbase/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/</code> directory.</p>
</div>
</div>
<div class="sect3">
<h4 id="chaos.monkey.properties"><a class="anchor" href="#chaos.monkey.properties"></a>182.5.5. Configuring Individual ChaosMonkey Actions</h4>
<div class="paragraph">
<p>ChaosMonkey integration tests can be configured per test run.
Create a Java properties file in the HBase CLASSPATH and pass it to ChaosMonkey using
the <code>-monkeyProps</code> configuration flag. Configurable properties, along with their default
values if applicable, are listed in the <code>org.apache.hadoop.hbase.chaos.factories.MonkeyConstants</code>
class. For properties that have defaults, you can override them by including them
in your properties file.</p>
</div>
<div class="paragraph">
<p>The following example uses a properties file called <a href="#monkey.properties">monkey.properties</a>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ bin/hbase org.apache.hadoop.hbase.IntegrationTestIngest -m slowDeterministic -monkeyProps monkey.properties</code></pre>
</div>
</div>
<div class="paragraph">
<p>The above command will start the integration tests and chaos monkey. It will look for the
properties file <em>monkey.properties</em> on the HBase CLASSPATH; e.g. inside the HBASE <em>conf</em> dir.</p>
</div>
<div class="paragraph">
<p>Here is an example chaos monkey file:</p>
</div>
<div id="monkey.properties" class="listingblock">
<div class="title">Example ChaosMonkey Properties File</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">sdm.action1.period=<span class="integer">120000</span>
sdm.action2.period=<span class="integer">40000</span>
move.regions.sleep.time=<span class="integer">80000</span>
move.regions.max.time=<span class="integer">1000000</span>
move.regions.sleep.time=<span class="integer">80000</span>
batch.restart.rs.ratio=<span class="float">0.4f</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Periods/time are expressed in milliseconds.</p>
</div>
<div class="paragraph">
<p>HBase 1.0.2 and newer adds the ability to restart HBase&#8217;s underlying ZooKeeper quorum or
HDFS nodes. To use these actions, you need to configure some new properties, which
have no reasonable defaults because they are deployment-specific, in your ChaosMonkey
properties file, which may be <code>hbase-site.xml</code> or a different properties file.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.it.clustermanager.hadoop.home<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$HADOOP_HOME<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.it.clustermanager.zookeeper.home<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$ZOOKEEPER_HOME<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.it.clustermanager.hbase.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hbase<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.it.clustermanager.hadoop.hdfs.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>hdfs<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.it.clustermanager.zookeeper.user<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>zookeeper<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="developing"><a class="anchor" href="#developing"></a>183. Developer Guidelines</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_branches"><a class="anchor" href="#_branches"></a>183.1. Branches</h3>
<div class="paragraph">
<p>We use Git for source code management and latest development happens on <code>master</code> branch. There are
branches for past major/minor/maintenance releases and important features and bug fixes are often
back-ported to them.</p>
</div>
</div>
<div class="sect2">
<h3 id="_policy_for_fix_version_in_jira"><a class="anchor" href="#_policy_for_fix_version_in_jira"></a>183.2. Policy for Fix Version in JIRA</h3>
<div class="paragraph">
<p>To determine if a given fix is in a given release purely from the release numbers following rules
are defined:</p>
</div>
<div class="paragraph">
<p>Fix version of X.Y.Z &#8658; fixed in all releases X.Y.Z' (where Z' = Z).<br>
Fix version of X.Y.0 &#8658; fixed in all releases X.Y'.* (where Y' = Y).<br>
Fix version of X.0.0 &#8658; fixed in all releases X'.*.* (where X' = X).<br></p>
</div>
<div class="paragraph">
<p>By this policy, fix version of 1.3.0 implies 1.4.0, but 1.3.2 does not imply 1.4.0 as we could not
tell purely from the numbers which release came first.</p>
</div>
</div>
<div class="sect2">
<h3 id="code.standards"><a class="anchor" href="#code.standards"></a>183.3. Code Standards</h3>
<div class="sect3">
<h4 id="_interface_classifications"><a class="anchor" href="#_interface_classifications"></a>183.3.1. Interface Classifications</h4>
<div class="paragraph">
<p>Interfaces are classified both by audience and by stability level.
These labels appear at the head of a class.
The conventions followed by HBase are inherited by its parent project, Hadoop.</p>
</div>
<div class="paragraph">
<p>The following interface classifications are commonly used:</p>
</div>
<div class="dlist">
<div class="title">InterfaceAudience</div>
<dl>
<dt class="hdlist1"><code>@InterfaceAudience.Public</code></dt>
<dd>
<p>APIs for users and HBase applications.
These APIs will be deprecated through major versions of HBase.</p>
</dd>
<dt class="hdlist1"><code>@InterfaceAudience.Private</code></dt>
<dd>
<p>APIs for HBase internals developers.
No guarantees on compatibility or availability in future versions.
Private interfaces do not need an <code>@InterfaceStability</code> classification.</p>
</dd>
<dt class="hdlist1"><code>@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)</code></dt>
<dd>
<p>APIs for HBase coprocessor writers.</p>
</dd>
<dt class="hdlist1">No <code>@InterfaceAudience</code> Classification</dt>
<dd>
<p>Packages without an <code>@InterfaceAudience</code> label are considered private.
Mark your new packages if publicly accessible.</p>
</dd>
</dl>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Excluding Non-Public Interfaces from API Documentation</div>
<div class="paragraph">
<p>Only interfaces classified <code>@InterfaceAudience.Public</code> should be included in API documentation (Javadoc). Committers must add new package excludes <code>ExcludePackageNames</code> section of the <em>pom.xml</em> for new packages which do not contain public classes.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<div class="title">@InterfaceStability</div>
<p><code>@InterfaceStability</code> is important for packages marked <code>@InterfaceAudience.Public</code>.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1"><code>@InterfaceStability.Stable</code></dt>
<dd>
<p>Public packages marked as stable cannot be changed without a deprecation path or a very good reason.</p>
</dd>
<dt class="hdlist1"><code>@InterfaceStability.Unstable</code></dt>
<dd>
<p>Public packages marked as unstable can be changed without a deprecation path.</p>
</dd>
<dt class="hdlist1"><code>@InterfaceStability.Evolving</code></dt>
<dd>
<p>Public packages marked as evolving may be changed, but it is discouraged.</p>
</dd>
<dt class="hdlist1">No <code>@InterfaceStability</code> Label</dt>
<dd>
<p>Public classes with no <code>@InterfaceStability</code> label are discouraged, and should be considered implicitly unstable.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>If you are unclear about how to mark packages, ask on the development list.</p>
</div>
</div>
<div class="sect3">
<h4 id="common.patch.feedback"><a class="anchor" href="#common.patch.feedback"></a>183.3.2. Code Formatting Conventions</h4>
<div class="paragraph">
<p>Please adhere to the following guidelines so that your patches can be reviewed more quickly.
These guidelines have been developed based upon common feedback on patches from new contributors.</p>
</div>
<div class="paragraph">
<p>See the <a href="http://www.oracle.com/technetwork/java/index-135089.html">Code
Conventions for the Java Programming Language</a> for more information on coding conventions in Java.
See <a href="#eclipse.code.formatting">eclipse.code.formatting</a> to setup Eclipse to check for some of
these guidelines automatically.</p>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.space.invaders"><a class="anchor" href="#common.patch.feedback.space.invaders"></a>Space Invaders</h5>
<div class="paragraph">
<p>Do not use extra spaces around brackets.
Use the second style, rather than the first.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">if</span> ( foo.equals( bar ) ) { <span class="comment">// don't do this</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">if</span> (foo.equals(bar)) {</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">foo = barArray[ i ]; <span class="comment">// don't do this</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">foo = barArray[i];</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.autogen"><a class="anchor" href="#common.patch.feedback.autogen"></a>Auto Generated Code</h5>
<div class="paragraph">
<p>Auto-generated code in Eclipse often uses bad variable names such as <code>arg0</code>.
Use more informative variable names.
Use code like the second example here.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> <span class="directive">public</span> <span class="type">void</span> readFields(<span class="predefined-type">DataInput</span> arg0) <span class="directive">throws</span> <span class="exception">IOException</span> { <span class="comment">// don't do this</span>
foo = arg0.readUTF(); <span class="comment">// don't do this</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> <span class="directive">public</span> <span class="type">void</span> readFields(<span class="predefined-type">DataInput</span> di) <span class="directive">throws</span> <span class="exception">IOException</span> {
foo = di.readUTF();</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.longlines"><a class="anchor" href="#common.patch.feedback.longlines"></a>Long Lines</h5>
<div class="paragraph">
<p>Keep lines less than 100 characters.
You can configure your IDE to do this automatically.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Bar bar = foo.veryLongMethodWithManyArguments(argument1, argument2, argument3, argument4, argument5, argument6, argument7, argument8, argument9); <span class="comment">// don't do this</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Bar bar = foo.veryLongMethodWithManyArguments(
argument1, argument2, argument3,argument4, argument5, argument6, argument7, argument8, argument9);</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.trailingspaces"><a class="anchor" href="#common.patch.feedback.trailingspaces"></a>Trailing Spaces</h5>
<div class="paragraph">
<p>Be sure there is a line break after the end of your code, and avoid lines with nothing but whitespace.
This makes diffs more meaningful.
You can configure your IDE to help with this.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Bar bar = foo.getBar(); &lt;--- imagine there is an extra space(s) after the semicolon.</code></pre>
</div>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.javadoc"><a class="anchor" href="#common.patch.feedback.javadoc"></a>API Documentation (Javadoc)</h5>
<div class="paragraph">
<p>Don&#8217;t forget Javadoc!</p>
</div>
<div class="paragraph">
<p>Javadoc warnings are checked during precommit.
If the precommit tool gives you a '-1', please fix the javadoc issue.
Your patch won&#8217;t be committed if it adds such warnings.</p>
</div>
<div class="paragraph">
<p>Also, no <code>@author</code> tags - that&#8217;s a rule.</p>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.findbugs"><a class="anchor" href="#common.patch.feedback.findbugs"></a>Findbugs</h5>
<div class="paragraph">
<p><code>Findbugs</code> is used to detect common bugs pattern.
It is checked during the precommit build.
If errors are found, please fix them.
You can run findbugs locally with <code>mvn
findbugs:findbugs</code>, which will generate the <code>findbugs</code> files locally.
Sometimes, you may have to write code smarter than <code>findbugs</code>.
You can annotate your code to tell <code>findbugs</code> you know what you&#8217;re doing, by annotating your class with the following annotation:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="annotation">@edu</span>.umd.cs.findbugs.annotations.SuppressWarnings(
value=<span class="string"><span class="delimiter">&quot;</span><span class="content">HE_EQUALS_USE_HASHCODE</span><span class="delimiter">&quot;</span></span>,
justification=<span class="string"><span class="delimiter">&quot;</span><span class="content">I know what I'm doing</span><span class="delimiter">&quot;</span></span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>It is important to use the Apache-licensed version of the annotations. That generally means using
annotations in the <code>edu.umd.cs.findbugs.annotations</code> package so that we can rely on the cleanroom
reimplementation rather than annotations in the <code>javax.annotations</code> package.</p>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.javadoc.defaults"><a class="anchor" href="#common.patch.feedback.javadoc.defaults"></a>Javadoc - Useless Defaults</h5>
<div class="paragraph">
<p>Don&#8217;t just leave javadoc tags the way IDE generates them, or fill redundant information in them.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> <span class="comment">/**
* @param table &lt;---- don't leave them empty!
* @param region An HRegion object. &lt;---- don't fill redundant information!
* @return Foo Object foo just created. &lt;---- Not useful information
* @throws SomeException &lt;---- Not useful. Function declarations already tell that!
* @throws BarException when something went wrong &lt;---- really?
*/</span>
<span class="directive">public</span> Foo createFoo(Bar bar);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Either add something descriptive to the tags, or just remove them.
The preference is to add something descriptive and useful.</p>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.onething"><a class="anchor" href="#common.patch.feedback.onething"></a>One Thing At A Time, Folks</h5>
<div class="paragraph">
<p>If you submit a patch for one thing, don&#8217;t do auto-reformatting or unrelated reformatting of code on a completely different area of code.</p>
</div>
<div class="paragraph">
<p>Likewise, don&#8217;t add unrelated cleanup or refactorings outside the scope of your Jira.</p>
</div>
</div>
<div class="sect4">
<h5 id="common.patch.feedback.tests"><a class="anchor" href="#common.patch.feedback.tests"></a>Ambiguous Unit Tests</h5>
<div class="paragraph">
<p>Make sure that you&#8217;re clear about what you are testing in your unit tests and why.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_garbage_collection_conserving_guidelines"><a class="anchor" href="#_garbage_collection_conserving_guidelines"></a>183.3.3. Garbage-Collection Conserving Guidelines</h4>
<div class="paragraph">
<p>The following guidelines were borrowed from <a href="http://engineering.linkedin.com/performance/linkedin-feed-faster-less-jvm-garbage" class="bare">http://engineering.linkedin.com/performance/linkedin-feed-faster-less-jvm-garbage</a>.
Keep them in mind to keep preventable garbage collection to a minimum. Have a look
at the blog post for some great examples of how to refactor your code according to
these guidelines.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Be careful with Iterators</p>
</li>
<li>
<p>Estimate the size of a collection when initializing</p>
</li>
<li>
<p>Defer expression evaluation</p>
</li>
<li>
<p>Compile the regex patterns in advance</p>
</li>
<li>
<p>Cache it if you can</p>
</li>
<li>
<p>String Interns are useful but dangerous</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect2">
<h3 id="design.invariants"><a class="anchor" href="#design.invariants"></a>183.4. Invariants</h3>
<div class="paragraph">
<p>We don&#8217;t have many but what we have we list below.
All are subject to challenge of course but until then, please hold to the rules of the road.</p>
</div>
<div class="sect3">
<h4 id="design.invariants.zk.data"><a class="anchor" href="#design.invariants.zk.data"></a>183.4.1. No permanent state in ZooKeeper</h4>
<div class="paragraph">
<p>ZooKeeper state should transient (treat it like memory). If ZooKeeper state is deleted, hbase should be able to recover and essentially be in the same state.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>.Exceptions: There are currently a few exceptions that we need to fix around whether a table is enabled or disabled.</p>
</li>
<li>
<p>Replication data is currently stored only in ZooKeeper.
Deleting ZooKeeper data related to replication may cause replication to be disabled.
Do not delete the replication tree, <em>/hbase/replication/</em>.</p>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
Replication may be disrupted and data loss may occur if you delete the replication tree (<em>/hbase/replication/</em>) from ZooKeeper.
Follow progress on this issue at <a href="https://issues.apache.org/jira/browse/HBASE-10295">HBASE-10295</a>.
</td>
</tr>
</table>
</div>
</li>
</ul>
</div>
</div>
</div>
<div class="sect2">
<h3 id="run.insitu"><a class="anchor" href="#run.insitu"></a>183.5. Running In-Situ</h3>
<div class="paragraph">
<p>If you are developing Apache HBase, frequently it is useful to test your changes against a more-real cluster than what you find in unit tests.
In this case, HBase can be run directly from the source in local-mode.
All you need to do is run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">${HBASE_HOME}/bin/start-hbase.sh</code></pre>
</div>
</div>
<div class="paragraph">
<p>This will spin up a full local-cluster, just as if you had packaged up HBase and installed it on your machine.</p>
</div>
<div class="paragraph">
<p>Keep in mind that you will need to have installed HBase into your local maven repository for the in-situ cluster to work properly.
That is, you will need to run:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">mvn clean install -DskipTests</code></pre>
</div>
</div>
<div class="paragraph">
<p>to ensure that maven can find the correct classpath and dependencies.
Generally, the above command is just a good thing to try running first, if maven is acting oddly.</p>
</div>
</div>
<div class="sect2">
<h3 id="add.metrics"><a class="anchor" href="#add.metrics"></a>183.6. Adding Metrics</h3>
<div class="paragraph">
<p>After adding a new feature a developer might want to add metrics.
HBase exposes metrics using the Hadoop Metrics 2 system, so adding a new metric involves exposing that metric to the hadoop system.
Unfortunately the API of metrics2 changed from hadoop 1 to hadoop 2.
In order to get around this a set of interfaces and implementations have to be loaded at runtime.
To get an in-depth look at the reasoning and structure of these classes you can read the blog post located <a href="https://blogs.apache.org/hbase/entry/migration_to_the_new_metrics">here</a>.
To add a metric to an existing MBean follow the short guide below:</p>
</div>
<div class="sect3">
<h4 id="_add_metric_name_and_function_to_hadoop_compat_interface"><a class="anchor" href="#_add_metric_name_and_function_to_hadoop_compat_interface"></a>183.6.1. Add Metric name and Function to Hadoop Compat Interface.</h4>
<div class="paragraph">
<p>Inside of the source interface the corresponds to where the metrics are generated (eg MetricsMasterSource for things coming from HMaster) create new static strings for metric name and description.
Then add a new method that will be called to add new reading.</p>
</div>
</div>
<div class="sect3">
<h4 id="_add_the_implementation_to_both_hadoop_1_and_hadoop_2_compat_modules"><a class="anchor" href="#_add_the_implementation_to_both_hadoop_1_and_hadoop_2_compat_modules"></a>183.6.2. Add the Implementation to Both Hadoop 1 and Hadoop 2 Compat modules.</h4>
<div class="paragraph">
<p>Inside of the implementation of the source (eg.
MetricsMasterSourceImpl in the above example) create a new histogram, counter, gauge, or stat in the init method.
Then in the method that was added to the interface wire up the parameter passed in to the histogram.</p>
</div>
<div class="paragraph">
<p>Now add tests that make sure the data is correctly exported to the metrics 2 system.
For this the MetricsAssertHelper is provided.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="git.best.practices"><a class="anchor" href="#git.best.practices"></a>183.7. Git Best Practices</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">Avoid git merges.</dt>
<dd>
<p>Use <code>git pull --rebase</code> or <code>git fetch</code> followed by <code>git rebase</code>.</p>
</dd>
<dt class="hdlist1">Do not use <code>git push --force</code>.</dt>
<dd>
<p>If the push does not work, fix the problem or ask for help.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>Please contribute to this document if you think of other Git best practices.</p>
</div>
<div class="sect3">
<h4 id="__code_rebase_all_git_branches_sh_code"><a class="anchor" href="#__code_rebase_all_git_branches_sh_code"></a>183.7.1. <code>rebase_all_git_branches.sh</code></h4>
<div class="paragraph">
<p>The <em>dev-support/rebase_all_git_branches.sh</em> script is provided to help keep your Git repository clean.
Use the <code>-h</code> parameter to get usage instructions.
The script automatically refreshes your tracking branches, attempts an automatic rebase of each local branch against its remote branch, and gives you the option to delete any branch which represents a closed <code>HBASE-</code> JIRA.
The script has one optional configuration option, the location of your Git directory.
You can set a default by editing the script.
Otherwise, you can pass the git directory manually by using the <code>-d</code> parameter, followed by an absolute or relative directory name, or even '.' for the current working directory.
The script checks the directory for sub-directory called <em>.git/</em>, before proceeding.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="submitting.patches"><a class="anchor" href="#submitting.patches"></a>183.8. Submitting Patches</h3>
<div class="paragraph">
<p>If you are new to submitting patches to open source or new to submitting patches to Apache, start by
reading the <a href="https://commons.apache.org/patches.html">On Contributing Patches</a> page from
<a href="https://commons.apache.org/">Apache Commons Project</a>.
It provides a nice overview that applies equally to the Apache HBase Project.</p>
</div>
<div class="sect3">
<h4 id="submitting.patches.create"><a class="anchor" href="#submitting.patches.create"></a>183.8.1. Create Patch</h4>
<div class="paragraph">
<p>Make sure you review <a href="#common.patch.feedback">common.patch.feedback</a> for code style. If your
patch
was generated incorrectly or your code does not adhere to the code formatting guidelines, you may
be asked to redo some work.</p>
</div>
<div class="listingblock">
<div class="title">Using submit-patch.py (recommended)</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ dev-support/submit-patch.py -jid HBASE-xxxxx</code></pre>
</div>
</div>
<div class="paragraph">
<p>Use this script to create patches, upload to jira and optionally create/update reviews on
Review Board. Patch name is automatically formatted as <em>(JIRA).(branch name).(patch number).patch</em>
to follow Yetus' naming rules. Use <code>-h</code> flag to know detailed usage information. Most useful options
are:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>-b BRANCH, --branch BRANCH</code> : Specify base branch for generating the diff. If not specified,
tracking branch is used. If there is no tracking branch, error will be thrown.</p>
</li>
<li>
<p><code>-jid JIRA_ID, --jira-id JIRA_ID</code> : If used, deduces next patch version from attachments in the
jira and uploads the new patch. Script will ask for jira username/password for authentication.
If not set, patch is named &lt;branch&gt;.patch.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>By default, it&#8217;ll also create/update review board. To skip that action, use <code>-srb</code> option. It uses
'Issue Links' in the jira to figure out if a review request already exists. If no review
request is present, then creates a new one and populates all required fields using jira summary,
patch description, etc. Also adds this review&#8217;s link to the jira.</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Save authentication credentials (optional)</dt>
<dd>
<p>Since attaching patches on JIRA and creating/changing review request on ReviewBoard requires
valid user authentication, the script will prompt you for username and password. To avoid the hassle every
time, set up <code>~/.apache-creds</code> with login details and encrypt it by following the steps in footer
of script&#8217;s help message.</p>
</dd>
<dt class="hdlist1">Python dependencies</dt>
<dd>
<p>To install required python dependencies, execute
<code>pip install -r dev-support/python-requirements.txt</code> from the master branch.</p>
</dd>
</dl>
</div>
<div class="olist arabic">
<div class="title">Manually</div>
<ol class="arabic">
<li>
<p>Use <code>git rebase -i</code> first, to combine (squash) smaller commits into a single larger one.</p>
</li>
<li>
<p>Create patch using IDE or Git commands. <code>git format-patch</code> is preferred since it preserves patch
author&#8217;s name and commit message. Also, it handles binary files by default, whereas <code>git diff</code>
ignores them unless you use the <code>--binary</code> option.</p>
</li>
<li>
<p>Patch name should be as follows to adhere to Yetus' naming convention:<br>
<code>(JIRA).(branch name).(patch number).patch</code><br>
For eg. HBASE-11625.master.001.patch, HBASE-XXXXX.branch-1.2.0005.patch, etc.</p>
</li>
<li>
<p>Attach the patch to the JIRA using <code>More&#8594;Attach Files</code> then click on <b class="button">Submit Patch</b>
button, which&#8217;ll trigger Hudson job to check patch for validity.</p>
</li>
<li>
<p>If your patch is longer than a single screen, also create a review on Review Board and
add the link to JIRA. See <a href="#reviewboard">reviewboard</a>.</p>
</li>
</ol>
</div>
<div class="ulist">
<div class="title">Few general guidelines</div>
<ul>
<li>
<p>Always patch against the master branch first, even if you want to patch in another branch.
HBase committers always apply patches first to the master branch, and backport if necessary.</p>
</li>
<li>
<p>Submit one single patch for a fix. If necessary, squash local commits to merge local commits into
a single one first. See this
<a href="http://stackoverflow.com/questions/5308816/how-to-use-git-merge-squash">Stack Overflow
question</a> for more information about squashing commits.</p>
</li>
<li>
<p>Please understand that not every patch may get committed, and that feedback will likely be
provided on the patch.</p>
</li>
<li>
<p>If you need to revise your patch, leave the previous patch file(s) attached to the JIRA, and
upload a new one with incremented patch number.<br>
Click on <b class="button">Cancel Patch</b> and then on <b class="button">Submit Patch</b> to trigger the presubmit run.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="submitting.patches.tests"><a class="anchor" href="#submitting.patches.tests"></a>183.8.2. Unit Tests</h4>
<div class="paragraph">
<p>Always add and/or update relevant unit tests when making the changes.
Make sure that new/changed unit tests pass locally before submitting the patch because it is faster
than waiting for presubmit result which runs full test suite. This will save your own time and
effort.
Use <a href="#mockito">mockito</a> to make mocks which are very useful for testing failure scenarios by
injecting appropriate failures.</p>
</div>
<div class="paragraph">
<p>If you are creating a new unit test class, notice how other unit test classes have
classification/sizing annotations before class name and a static methods for setup/teardown of
testing environment. Be sure to include annotations in any new unit test files.
See <a href="#hbase.tests">hbase.tests</a> for more information on tests.</p>
</div>
</div>
<div class="sect3">
<h4 id="_integration_tests"><a class="anchor" href="#_integration_tests"></a>183.8.3. Integration Tests</h4>
<div class="paragraph">
<p>Significant new features should provide an integration test in addition to unit tests, suitable for exercising the new feature at different points in its configuration space.</p>
</div>
</div>
<div class="sect3">
<h4 id="reviewboard"><a class="anchor" href="#reviewboard"></a>183.8.4. ReviewBoard</h4>
<div class="paragraph">
<p>Patches larger than one screen, or patches that will be tricky to review, should go through <a href="https://reviews.apache.org">ReviewBoard</a>.</p>
</div>
<div class="olist arabic">
<div class="title">Procedure: Use ReviewBoard</div>
<ol class="arabic">
<li>
<p>Register for an account if you don&#8217;t already have one.
It does not use the credentials from <a href="https://issues.apache.org">issues.apache.org</a>.
Log in.</p>
</li>
<li>
<p>Click <span class="label">New Review Request</span>.</p>
</li>
<li>
<p>Choose the <code>hbase-git</code> repository.
Click Choose File to select the diff and optionally a parent diff.
Click <b class="button">Create Review Request</b>.</p>
</li>
<li>
<p>Fill in the fields as required.
At the minimum, fill in the <span class="label">Summary</span> and choose <code>hbase</code> as the <span class="label">Review Group</span>.
If you fill in the <span class="label">Bugs</span> field, the review board links back to the relevant JIRA.
The more fields you fill in, the better.
Click <b class="button">Publish</b> to make your review request public.
An email will be sent to everyone in the <code>hbase</code> group, to review the patch.</p>
</li>
<li>
<p>Back in your JIRA, click , and paste in the URL of your ReviewBoard request.
This attaches the ReviewBoard to the JIRA, for easy access.</p>
</li>
<li>
<p>To cancel the request, click .</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>For more information on how to use ReviewBoard, see <a href="http://www.reviewboard.org/docs/manual/1.5/">the ReviewBoard
documentation</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="_guide_for_hbase_committers"><a class="anchor" href="#_guide_for_hbase_committers"></a>183.8.5. Guide for HBase Committers</h4>
<div class="sect4">
<h5 id="_becoming_a_committer"><a class="anchor" href="#_becoming_a_committer"></a>Becoming a committer</h5>
<div class="paragraph">
<p>Committers are responsible for reviewing and integrating code changes, testing
and voting on release candidates, weighing in on design discussions, as well as
other types of project contributions. The PMC votes to make a contributor a
committer based on an assessment of their contributions to the project. It is
expected that committers demonstrate a sustained history of high-quality
contributions to the project and community involvement.</p>
</div>
<div class="paragraph">
<p>Contributions can be made in many ways. There is no single path to becoming a
committer, nor any expected timeline. Submitting features, improvements, and bug
fixes is the most common avenue, but other methods are both recognized and
encouraged (and may be even more important to the health of HBase as a project and a
community). A non-exhaustive list of potential contributions (in no particular
order):</p>
</div>
<div class="ulist">
<ul>
<li>
<p><a href="#appendix_contributing_to_documentation">Update the documentation</a> for new
changes, best practices, recipes, and other improvements.</p>
</li>
<li>
<p>Keep the website up to date.</p>
</li>
<li>
<p>Perform testing and report the results. For instance, scale testing and
testing non-standard configurations is always appreciated.</p>
</li>
<li>
<p>Maintain the shared Jenkins testing environment and other testing
infrastructure.</p>
</li>
<li>
<p><a href="#hbase.rc.voting">Vote on release candidates</a> after performing validation, even if non-binding.
A non-binding vote is a vote by a non-committer.</p>
</li>
<li>
<p>Provide input for discussion threads on the <a href="/mail-lists.html">mailing lists</a> (which usually have
<code>[DISCUSS]</code> in the subject line).</p>
</li>
<li>
<p>Answer questions questions on the user or developer mailing lists and on
Slack.</p>
</li>
<li>
<p>Make sure the HBase community is a welcoming one and that we adhere to our
<a href="/coc.html">Code of conduct</a>. Alert the PMC if you
have concerns.</p>
</li>
<li>
<p>Review other people&#8217;s work (both code and non-code) and provide public
feedback.</p>
</li>
<li>
<p>Report bugs that are found, or file new feature requests.</p>
</li>
<li>
<p>Triage issues and keep JIRA organized. This includes closing stale issues,
labeling new issues, updating metadata, and other tasks as needed.</p>
</li>
<li>
<p>Mentor new contributors of all sorts.</p>
</li>
<li>
<p>Give talks and write blogs about HBase. Add these to the <a href="/">News</a> section
of the website.</p>
</li>
<li>
<p>Provide UX feedback about HBase, the web UI, the CLI, APIs, and the website.</p>
</li>
<li>
<p>Write demo applications and scripts.</p>
</li>
<li>
<p>Help attract and retain a diverse community.</p>
</li>
<li>
<p>Interact with other projects in ways that benefit HBase and those other
projects.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Not every individual is able to do all (or even any) of the items on this list.
If you think of other ways to contribute, go for it (and add them to the list).
A pleasant demeanor and willingness to contribute are all you need to make a
positive impact on the HBase project. Invitations to become a committer are the
result of steady interaction with the community over the long term, which builds
trust and recognition.</p>
</div>
</div>
<div class="sect4">
<h5 id="_new_committers"><a class="anchor" href="#_new_committers"></a>New committers</h5>
<div class="paragraph">
<p>New committers are encouraged to first read Apache&#8217;s generic committer
documentation:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><a href="https://www.apache.org/dev/new-committers-guide.html">Apache New Committer Guide</a></p>
</li>
<li>
<p><a href="https://www.apache.org/dev/committers.html">Apache Committer FAQ</a></p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="_review"><a class="anchor" href="#_review"></a>Review</h5>
<div class="paragraph">
<p>HBase committers should, as often as possible, attempt to review patches
submitted by others. Ideally every submitted patch will get reviewed by a
committer <em>within a few days</em>. If a committer reviews a patch they have not
authored, and believe it to be of sufficient quality, then they can commit the
patch. Otherwise the patch should be cancelled with a clear explanation for why
it was rejected.</p>
</div>
<div class="paragraph">
<p>The list of submitted patches is in the
<a href="https://issues.apache.org/jira/secure/IssueNavigator.jspa?mode=hide&amp;requestId=12312392">HBase Review Queue</a>,
which is ordered by time of last modification. Committers should scan the list
from top to bottom, looking for patches that they feel qualified to review and
possibly commit. If you see a patch you think someone else is better qualified
to review, you can mention them by username in the JIRA.</p>
</div>
<div class="paragraph">
<p>For non-trivial changes, it is required that another committer review your
patches before commit. <strong>Self-commits of non-trivial patches are not allowed.</strong>
Use the <b class="button">Submit Patch</b> button in JIRA, just like other contributors, and
then wait for a <code>+1</code> response from another committer before committing.</p>
</div>
</div>
<div class="sect4">
<h5 id="_reject"><a class="anchor" href="#_reject"></a>Reject</h5>
<div class="paragraph">
<p>Patches which do not adhere to the guidelines in
<a href="https://hbase.apache.org/book.html#developer">HowToContribute</a> and to the
<a href="https://wiki.apache.org/hadoop/CodeReviewChecklist">code review checklist</a>
should be rejected. Committers should always be polite to contributors and try
to instruct and encourage them to contribute better patches. If a committer
wishes to improve an unacceptable patch, then it should first be rejected, and a
new patch should be attached by the committer for further review.</p>
</div>
</div>
<div class="sect4">
<h5 id="committing.patches"><a class="anchor" href="#committing.patches"></a>Commit</h5>
<div class="paragraph">
<p>Committers commit patches to the Apache HBase GIT repository.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">Before you commit!!!!</div>
<div class="paragraph">
<p>Make sure your local configuration is correct, especially your identity and email.
Examine the output of the $ git config --list command and be sure it is correct.
See <a href="https://help.github.com/articles/set-up-git">Set Up Git</a> if you need
pointers.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>When you commit a patch:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Include the Jira issue ID in the commit message along with a short description
of the change. Try to add something more than just the Jira title so that
someone looking at <code>git log</code> output doesn&#8217;t have to go to Jira to discern what
the change is about. Be sure to get the issue ID right, because this causes
Jira to link to the change in Git (use the issue&#8217;s "All" tab to see these
automatic links).</p>
</li>
<li>
<p>Commit the patch to a new branch based off <code>master</code> or the other intended
branch. It&#8217;s a good idea to include the JIRA ID in the name of this branch.
Check out the relevant target branch where you want to commit, and make sure
your local branch has all remote changes, by doing a git pull --rebase or
another similar command. Next, cherry-pick the change into each relevant
branch (such as master), and push the changes to the remote branch using
a command such as git push &lt;remote-server&gt; &lt;remote-branch&gt;.</p>
<div class="admonitionblock warning">
<table>
<tr>
<td class="icon">
<i class="fa icon-warning" title="Warning"></i>
</td>
<td class="content">
If you do not have all remote changes, the push will fail.
If the push fails for any reason, fix the problem or ask for help.
Do not do a git push --force.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Before you can commit a patch, you need to determine how the patch was created.
The instructions and preferences around the way to create patches have changed,
and there will be a transition period.</p>
</div>
<div class="ulist">
<div class="title">Determine How a Patch Was Created</div>
<ul>
<li>
<p>If the first few lines of the patch look like the headers of an email, with a From, Date, and
Subject, it was created using git format-patch. This is the preferred way, because you can
reuse the submitter&#8217;s commit message. If the commit message is not appropriate, you can still use
the commit, then run <code>git commit --amend</code> and reword as appropriate.</p>
</li>
<li>
<p>If the first line of the patch looks similar to the following, it was created using git diff without <code>--no-prefix</code>.
This is acceptable too.
Notice the <code>a</code> and <code>b</code> in front of the file names.
This is the indication that the patch was not created with <code>--no-prefix</code>.</p>
<div class="listingblock">
<div class="content">
<pre>diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc</pre>
</div>
</div>
</li>
<li>
<p>If the first line of the patch looks similar to the following (without the <code>a</code> and <code>b</code>), the
patch was created with git diff --no-prefix and you need to add <code>-p0</code> to the git apply command
below.</p>
<div class="listingblock">
<div class="content">
<pre>diff --git src/main/asciidoc/_chapters/developer.adoc src/main/asciidoc/_chapters/developer.adoc</pre>
</div>
</div>
</li>
</ul>
</div>
<div class="exampleblock">
<div class="title">Example 46. Example of committing a Patch</div>
<div class="content">
<div class="paragraph">
<p>One thing you will notice with these examples is that there are a lot of
git pull commands. The only command that actually writes anything to the
remote repository is git push, and you need to make absolutely sure you have
the correct versions of everything and don&#8217;t have any conflicts before pushing.
The extra git pull commands are usually redundant, but better safe than sorry.</p>
</div>
<div class="paragraph">
<p>The first example shows how to apply a patch that was generated with git
format-patch and apply it to the <code>master</code> and <code>branch-1</code> branches.</p>
</div>
<div class="paragraph">
<p>The directive to use git format-patch rather than git diff, and not to use
<code>--no-prefix</code>, is a new one. See the second example for how to apply a patch
created with git diff, and educate the person who created the patch.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ git checkout -b HBASE-XXXX
$ git am ~/Downloads/HBASE-XXXX-v2.patch --signoff # If you are committing someone else's patch.
$ git checkout master
$ git pull --rebase
$ git cherry-pick &lt;sha-from-commit&gt;
# Resolve conflicts if necessary or ask the submitter to do it
$ git pull --rebase # Better safe than sorry
$ git push origin master
# Backport to branch-1
$ git checkout branch-1
$ git pull --rebase
$ git cherry-pick &lt;sha-from-commit&gt;
# Resolve conflicts if necessary
$ git pull --rebase # Better safe than sorry
$ git push origin branch-1
$ git branch -D HBASE-XXXX</pre>
</div>
</div>
<div class="paragraph">
<p>This example shows how to commit a patch that was created using git diff
without <code>--no-prefix</code>. If the patch was created with <code>--no-prefix</code>, add <code>-p0</code> to
the git apply command.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>$ git apply ~/Downloads/HBASE-XXXX-v2.patch
$ git commit -m "HBASE-XXXX Really Good Code Fix (Joe Schmo)" --author=&lt;contributor&gt; -a # This and next command is needed for patches created with 'git diff'
$ git commit --amend --signoff
$ git checkout master
$ git pull --rebase
$ git cherry-pick &lt;sha-from-commit&gt;
# Resolve conflicts if necessary or ask the submitter to do it
$ git pull --rebase # Better safe than sorry
$ git push origin master
# Backport to branch-1
$ git checkout branch-1
$ git pull --rebase
$ git cherry-pick &lt;sha-from-commit&gt;
# Resolve conflicts if necessary or ask the submitter to do it
$ git pull --rebase # Better safe than sorry
$ git push origin branch-1
$ git branch -D HBASE-XXXX</pre>
</div>
</div>
</div>
</div>
</li>
<li>
<p>Resolve the issue as fixed, thanking the contributor.
Always set the "Fix Version" at this point, but only set a single fix version
for each branch where the change was committed, the earliest release in that
branch in which the change will appear.</p>
</li>
</ol>
</div>
<div class="sect5">
<h6 id="_commit_message_format"><a class="anchor" href="#_commit_message_format"></a>Commit Message Format</h6>
<div class="paragraph">
<p>The commit message should contain the JIRA ID and a description of what the patch does.
The preferred commit message format is:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>&lt;jira-id&gt; &lt;jira-title&gt; (&lt;contributor-name-if-not-commit-author&gt;)</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>HBASE-12345 Fix All The Things (jane@example.com)</pre>
</div>
</div>
<div class="paragraph">
<p>If the contributor used git format-patch to generate the patch, their commit
message is in their patch and you can use that, but be sure the JIRA ID is at
the front of the commit message, even if the contributor left it out.</p>
</div>
</div>
<div class="sect5">
<h6 id="committer.amending.author"><a class="anchor" href="#committer.amending.author"></a>Use GitHub&#8217;s "Co-authored-by" when there are multiple authors</h6>
<div class="paragraph">
<p>We&#8217;ve established the practice of committing to master and then cherry picking back to branches whenever possible, unless</p>
</div>
<div class="ulist">
<ul>
<li>
<p>it&#8217;s breaking compat: In which case, if it can go in minor releases, backport to branch-1 and branch-2.</p>
</li>
<li>
<p>it&#8217;s a new feature: No for maintenance releases, For minor releases, discuss and arrive at consensus.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>There are occasions when there are multiple author for a patch.
For example when there is a minor conflict we can fix it up and just proceed with the commit.
The amending author will be different from the original committer, so you should also attribute to the original author by
adding one or more <code>Co-authored-by</code> trailers to the commit&#8217;s message.
See <a href="https://help.github.com/en/articles/creating-a-commit-with-multiple-authors/">the GitHub documentation for "Creating a commit with multiple authors"</a>.</p>
</div>
<div class="paragraph">
<p>In short, these are the steps to add Co-authors that will be tracked by GitHub:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Collect the name and email address for each co-author.</p>
</li>
<li>
<p>Commit the change, but after your commit description, instead of a closing quotation, add two empty lines. (Do not close the commit message with a quotation mark)</p>
</li>
<li>
<p>On the next line of the commit message, type <code>Co-authored-by: name &lt;<a href="mailto:name@example.com">name@example.com</a>&gt;</code>. After the co-author information, add a closing quotation mark.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>Here is the example from the GitHub page, using 2 Co-authors:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml">$ git commit -m &quot;Refactor usability tests.
<span class="error">&gt;</span>
<span class="error">&gt;</span>
Co-authored-by: name <span class="tag">&lt;name</span><span class="error">@</span><span class="attribute-name">example.com</span><span class="tag">&gt;</span>
Co-authored-by: another-name <span class="tag">&lt;another-name</span><span class="error">@</span><span class="attribute-name">example.com</span><span class="tag">&gt;</span>&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note: <code>Amending-Author: Author &lt;committer@apache&gt;</code> was used prior to this
<a href="https://lists.apache.org/thread.html/f00b5f9b65570e777dbb31c37d7b0ffc55c5fc567aefdb456608a042@%3Cdev.hbase.apache.org%3E">DISCUSSION</a>.</p>
</div>
</div>
<div class="sect5">
<h6 id="_close_related_github_prs"><a class="anchor" href="#_close_related_github_prs"></a>Close related GitHub PRs</h6>
<div class="paragraph">
<p>As a project we work to ensure there&#8217;s a JIRA associated with each change, but we don&#8217;t mandate any particular tool be used for reviews. Due to implementation details of the ASF&#8217;s integration between hosted git repositories and GitHub, the PMC has no ability to directly close PRs on our GitHub repo. In the event that a contributor makes a Pull Request on GitHub, either because the contributor finds that easier than attaching a patch to JIRA or because a reviewer prefers that UI for examining changes, it&#8217;s important to make note of the PR in the commit that goes to the master branch so that PRs are kept up to date.</p>
</div>
<div class="paragraph">
<p>To read more about the details of what kinds of commit messages will work with the GitHub "close via keyword in commit" mechanism see <a href="https://help.github.com/articles/closing-issues-using-keywords/">the GitHub documentation for "Closing issues using keywords"</a>. In summary, you should include a line with the phrase "closes #XXX", where the XXX is the pull request id. The pull request id is usually given in the GitHub UI in grey at the end of the subject heading.</p>
</div>
</div>
<div class="sect5">
<h6 id="committer.tests"><a class="anchor" href="#committer.tests"></a>Committers are responsible for making sure commits do not break the build or tests</h6>
<div class="paragraph">
<p>If a committer commits a patch, it is their responsibility to make sure it passes the test suite.
It is helpful if contributors keep an eye out that their patch does not break the hbase build and/or tests, but ultimately, a contributor cannot be expected to be aware of all the particular vagaries and interconnections that occur in a project like HBase.
A committer should.</p>
</div>
</div>
<div class="sect5">
<h6 id="git.patch.flow"><a class="anchor" href="#git.patch.flow"></a>Patching Etiquette</h6>
<div class="paragraph">
<p>In the thread <a href="http://search-hadoop.com/m/DHED4EiwOz">HBase, mail # dev - ANNOUNCEMENT: Git Migration In Progress (WAS &#8658;
Re: Git Migration)</a>, it was agreed on the following patch flow</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Develop and commit the patch against master first.</p>
</li>
<li>
<p>Try to cherry-pick the patch when backporting if possible.</p>
</li>
<li>
<p>If this does not work, manually commit the patch to the branch.</p>
</li>
</ol>
</div>
</div>
<div class="sect5">
<h6 id="_merge_commits"><a class="anchor" href="#_merge_commits"></a>Merge Commits</h6>
<div class="paragraph">
<p>Avoid merge commits, as they create problems in the git history.</p>
</div>
</div>
<div class="sect5">
<h6 id="_committing_documentation"><a class="anchor" href="#_committing_documentation"></a>Committing Documentation</h6>
<div class="paragraph">
<p>See <a href="#appendix_contributing_to_documentation">appendix contributing to documentation</a>.</p>
</div>
</div>
<div class="sect5">
<h6 id="_how_to_re_trigger_github_pull_request_checks_re_build"><a class="anchor" href="#_how_to_re_trigger_github_pull_request_checks_re_build"></a>How to re-trigger github Pull Request checks/re-build</h6>
<div class="paragraph">
<p>A Pull Request (PR) submission triggers the hbase yetus checks. The checks make
sure the patch doesn&#8217;t break the build or introduce test failures. The checks take
around four hours to run (They are the same set run when you submit a patch via
HBASE JIRA). When finished, they add a report to the PR as a comment. If a problem
w/ the patch&#8201;&#8212;&#8201;failed compile, checkstyle violation, or an added findbugs&#8201;&#8212;&#8201;the original author makes fixes and pushes a new patch. This re-runs the checks
to produce a new report.</p>
</div>
<div class="paragraph">
<p>Sometimes though, the patch is good but a flakey, unrelated test has the report vote -1
on the patch. In this case, <strong>committers</strong> can retrigger the check run by doing a force push of the
exact same patch. Or, click on the <code>Console output</code> link which shows toward the end
of the report (For example <code><a href="https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-289/1/console" class="bare">https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-289/1/console</a></code>).
This will take you to <code>builds.apache.org</code>, to the build run that failed. See the
"breadcrumbs" along the top (where breadcrumbs is the listing of the directories that
gets us to this particular build page). It&#8217;ll look something like
<code>Jenkins &gt; HBase-PreCommit-GitHub-PR &gt; PR-289 &gt; #1</code>. Click on the
PR number&#8201;&#8212;&#8201;i.e. PR-289 in our example&#8201;&#8212;&#8201;and then, when you&#8217;ve arrived at the PR page,
find the 'Build with Parameters' menu-item (along top left-hand menu). Click here and
then <code>Build</code> leaving the JIRA_ISSUE_KEY empty. This will re-run your checks.</p>
</div>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_dialog"><a class="anchor" href="#_dialog"></a>183.8.6. Dialog</h4>
<div class="paragraph">
<p>Committers should hang out in the #hbase room on irc.freenode.net for real-time discussions.
However any substantive discussion (as with any off-list project-related discussion) should be re-iterated in Jira or on the developer list.</p>
</div>
</div>
<div class="sect3">
<h4 id="_do_not_edit_jira_comments"><a class="anchor" href="#_do_not_edit_jira_comments"></a>183.8.7. Do not edit JIRA comments</h4>
<div class="paragraph">
<p>Misspellings and/or bad grammar is preferable to the disruption a JIRA comment edit causes: See the discussion at <a href="http://search-hadoop.com/?q=%5BReopened%5D+%28HBASE-451%29+Remove+HTableDescriptor+from+HRegionInfo&amp;fc_project=HBase">Re:(HBASE-451) Remove HTableDescriptor from HRegionInfo</a></p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="thirdparty"><a class="anchor" href="#thirdparty"></a>183.9. The hbase-thirdparty dependency and shading/relocation</h3>
<div class="paragraph">
<p>A new project was created for the release of hbase-2.0.0. It was called
<code>hbase-thirdparty</code>. This project exists only to provide the main hbase
project with relocated&#8201;&#8212;&#8201;or shaded&#8201;&#8212;&#8201;versions of popular thirdparty
libraries such as guava, netty, and protobuf. The mainline HBase project
relies on the relocated versions of these libraries gotten from hbase-thirdparty
rather than on finding these classes in their usual locations. We do this so
we can specify whatever the version we wish. If we don&#8217;t relocate, we must
harmonize our version to match that which hadoop, spark, and other projects use.</p>
</div>
<div class="paragraph">
<p>For developers, this means you need to be careful referring to classes from
netty, guava, protobuf, gson, etc. (see the hbase-thirdparty pom.xml for what
it provides). Devs must refer to the hbase-thirdparty provided classes. In
practice, this is usually not an issue (though it can be a bit of a pain). You
will have to hunt for the relocated version of your particular class. You&#8217;ll
find it by prepending the general relocation prefix of <code>org.apache.hbase.thirdparty.</code>.
For example if you are looking for <code>com.google.protobuf.Message</code>, the relocated
version used by HBase internals can be found at
<code>org.apache.hbase.thirdparty.com.google.protobuf.Message</code>.</p>
</div>
<div class="paragraph">
<p>For a few thirdparty libs, like protobuf (see the protobuf chapter in this book
for the why), your IDE may give you both options&#8201;&#8212;&#8201;the <code>com.google.protobuf.<strong></code>
and the <code>org.apache.hbase.thirdparty.com.google.protobuf.</strong></code>&#8201;&#8212;&#8201;because both
classes are on your CLASSPATH. Unless you are doing the particular juggling
required in Coprocessor Endpoint development (again see above cited protobuf
chapter), you&#8217;ll want to use the shaded version, always.</p>
</div>
<div class="paragraph">
<p>The <code>hbase-thirdparty</code> project has groupid of <code>org.apache.hbase.thirdparty</code>.
As of this writing, it provides three jars; one for netty with an artifactid of
<code>hbase-thirdparty-netty</code>, one for protobuf at <code>hbase-thirdparty-protobuf</code> and then
a jar for all else&#8201;&#8212;&#8201;gson, guava&#8201;&#8212;&#8201;at <code>hbase-thirdpaty-miscellaneous</code>.</p>
</div>
<div class="paragraph">
<p>The hbase-thirdparty artifacts are a product produced by the Apache HBase
project under the aegis of the HBase Project Management Committee. Releases
are done via the usual voting project on the hbase dev mailing list. If issue
in the hbase-thirdparty, use the hbase JIRA and mailing lists to post notice.</p>
</div>
</div>
<div class="sect2">
<h3 id="hbase.archetypes.development"><a class="anchor" href="#hbase.archetypes.development"></a>183.10. Development of HBase-related Maven archetypes</h3>
<div class="paragraph">
<p>The development of HBase-related Maven archetypes was begun with
<a href="https://issues.apache.org/jira/browse/HBASE-14876">HBASE-14876</a>.
For an overview of the hbase-archetypes infrastructure and instructions
for developing new HBase-related Maven archetypes, please see
<code>hbase/hbase-archetypes/README.md</code>.</p>
</div>
</div>
</div>
</div>
<h1 id="unit.tests" class="sect0"><a class="anchor" href="#unit.tests"></a>Unit Testing HBase Applications</h1>
<div class="openblock partintro">
<div class="content">
This chapter discusses unit testing your HBase application using JUnit, Mockito, MRUnit, and HBaseTestingUtility.
Much of the information comes from <a href="http://blog.cloudera.com/blog/2013/09/how-to-test-hbase-applications-using-popular-tools/">a community blog post about testing HBase applications</a>.
For information on unit tests for HBase itself, see <a href="#hbase.tests">hbase.tests</a>.
</div>
</div>
<div class="sect1">
<h2 id="_junit"><a class="anchor" href="#_junit"></a>184. JUnit</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase uses <a href="http://junit.org">JUnit</a> for unit tests</p>
</div>
<div class="paragraph">
<p>This example will add unit tests to the following example class:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">MyHBaseDAO</span> {
<span class="directive">public</span> <span class="directive">static</span> <span class="type">void</span> insertRecord(Table.getTable(table), HBaseTestObj obj)
<span class="directive">throws</span> <span class="exception">Exception</span> {
Put put = createPut(obj);
table.put(put);
}
<span class="directive">private</span> <span class="directive">static</span> Put createPut(HBaseTestObj obj) {
Put put = <span class="keyword">new</span> Put(Bytes.toBytes(obj.getRowKey()));
put.add(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>),
Bytes.toBytes(obj.getData1()));
put.add(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-2</span><span class="delimiter">&quot;</span></span>),
Bytes.toBytes(obj.getData2()));
<span class="keyword">return</span> put;
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>The first step is to add JUnit dependencies to your Maven POM file:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>junit<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>junit<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>4.11<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;scope&gt;</span>test<span class="tag">&lt;/scope&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Next, add some unit tests to your code.
Tests are annotated with <code>@Test</code>.
Here, the unit tests are in bold.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">TestMyHbaseDAOData</span> {
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testCreatePut() <span class="directive">throws</span> <span class="exception">Exception</span> {
HBaseTestObj obj = <span class="keyword">new</span> HBaseTestObj();
obj.setRowKey(<span class="string"><span class="delimiter">&quot;</span><span class="content">ROWKEY-1</span><span class="delimiter">&quot;</span></span>);
obj.setData1(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-1</span><span class="delimiter">&quot;</span></span>);
obj.setData2(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-2</span><span class="delimiter">&quot;</span></span>);
Put put = MyHBaseDAO.createPut(obj);
assertEquals(obj.getRowKey(), Bytes.toString(put.getRow()));
assertEquals(obj.getData1(), Bytes.toString(put.get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>)).get(<span class="integer">0</span>).getValue()));
assertEquals(obj.getData2(), Bytes.toString(put.get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-2</span><span class="delimiter">&quot;</span></span>)).get(<span class="integer">0</span>).getValue()));
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>These tests ensure that your <code>createPut</code> method creates, populates, and returns a <code>Put</code> object with expected values.
Of course, JUnit can do much more than this.
For an introduction to JUnit, see <a href="https://github.com/junit-team/junit/wiki/Getting-started" class="bare">https://github.com/junit-team/junit/wiki/Getting-started</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="mockito"><a class="anchor" href="#mockito"></a>185. Mockito</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Mockito is a mocking framework.
It goes further than JUnit by allowing you to test the interactions between objects without having to replicate the entire environment.
You can read more about Mockito at its project site, <a href="https://code.google.com/p/mockito/" class="bare">https://code.google.com/p/mockito/</a>.</p>
</div>
<div class="paragraph">
<p>You can use Mockito to do unit testing on smaller units.
For instance, you can mock a <code>org.apache.hadoop.hbase.Server</code> instance or a <code>org.apache.hadoop.hbase.master.MasterServices</code> interface reference rather than a full-blown <code>org.apache.hadoop.hbase.master.HMaster</code>.</p>
</div>
<div class="paragraph">
<p>This example builds upon the example code in <a href="#unit.tests">unit.tests</a>, to test the <code>insertRecord</code> method.</p>
</div>
<div class="paragraph">
<p>First, add a dependency for Mockito to your Maven POM file.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.mockito<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>mockito-core<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>2.1.0<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;scope&gt;</span>test<span class="tag">&lt;/scope&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Next, add a <code>@RunWith</code> annotation to your test class, to direct it to use Mockito.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="annotation">@RunWith</span>(MockitoJUnitRunner.class)
<span class="directive">public</span> <span class="type">class</span> <span class="class">TestMyHBaseDAO</span>{
<span class="annotation">@Mock</span>
<span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
<span class="annotation">@Mock</span>
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(config);
<span class="annotation">@Mock</span>
<span class="directive">private</span> Table table;
<span class="annotation">@Captor</span>
<span class="directive">private</span> ArgumentCaptor putCaptor;
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testInsertRecord() <span class="directive">throws</span> <span class="exception">Exception</span> {
<span class="comment">//return mock table when getTable is called</span>
when(connection.getTable(TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">tablename</span><span class="delimiter">&quot;</span></span>)).thenReturn(table);
<span class="comment">//create test object and make a call to the DAO that needs testing</span>
HBaseTestObj obj = <span class="keyword">new</span> HBaseTestObj();
obj.setRowKey(<span class="string"><span class="delimiter">&quot;</span><span class="content">ROWKEY-1</span><span class="delimiter">&quot;</span></span>);
obj.setData1(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-1</span><span class="delimiter">&quot;</span></span>);
obj.setData2(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-2</span><span class="delimiter">&quot;</span></span>);
MyHBaseDAO.insertRecord(table, obj);
verify(table).put(putCaptor.capture());
Put put = putCaptor.getValue();
assertEquals(Bytes.toString(put.getRow()), obj.getRowKey());
<span class="keyword">assert</span>(put.has(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>)));
<span class="keyword">assert</span>(put.has(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>), Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-2</span><span class="delimiter">&quot;</span></span>)));
assertEquals(Bytes.toString(put.get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>),Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>)).get(<span class="integer">0</span>).getValue()), <span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-1</span><span class="delimiter">&quot;</span></span>);
assertEquals(Bytes.toString(put.get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>),Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-2</span><span class="delimiter">&quot;</span></span>)).get(<span class="integer">0</span>).getValue()), <span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-2</span><span class="delimiter">&quot;</span></span>);
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>This code populates <code>HBaseTestObj</code> with <code>ROWKEY-1'', </code>DATA-1'', ``DATA-2'' as values.
It then inserts the record into the mocked table.
The Put that the DAO would have inserted is captured, and values are tested to verify that they are what you expected them to be.</p>
</div>
<div class="paragraph">
<p>The key here is to manage Connection and Table instance creation outside the DAO.
This allows you to mock them cleanly and test Puts as shown above.
Similarly, you can now expand into other operations such as Get, Scan, or Delete.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_mrunit"><a class="anchor" href="#_mrunit"></a>186. MRUnit</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="https://mrunit.apache.org/">Apache MRUnit</a> is a library that allows you to unit-test MapReduce jobs.
You can use it to test HBase jobs in the same way as other MapReduce jobs.</p>
</div>
<div class="paragraph">
<p>Given a MapReduce job that writes to an HBase table called <code>MyTest</code>, which has one column family called <code>CF</code>, the reducer of such a job could look like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">MyReducer</span> <span class="directive">extends</span> TableReducer&lt;Text, Text, ImmutableBytesWritable&gt; {
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="directive">static</span> <span class="directive">final</span> <span class="type">byte</span><span class="type">[]</span> QUALIFIER = <span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="directive">public</span> <span class="type">void</span> reduce(Text key, <span class="predefined-type">Iterable</span>&lt;Text&gt; values, <span class="predefined-type">Context</span> context) <span class="directive">throws</span> <span class="exception">IOException</span>, <span class="exception">InterruptedException</span> {
<span class="comment">//bunch of processing to extract data to be inserted, in our case, let's say we are simply</span>
<span class="comment">//appending all the records we receive from the mapper for this particular</span>
<span class="comment">//key and insert one record into HBase</span>
<span class="predefined-type">StringBuffer</span> data = <span class="keyword">new</span> <span class="predefined-type">StringBuffer</span>();
Put put = <span class="keyword">new</span> Put(Bytes.toBytes(key.toString()));
<span class="keyword">for</span> (Text val : values) {
data = data.append(val);
}
put.add(CF, QUALIFIER, Bytes.toBytes(data.toString()));
<span class="comment">//write to HBase</span>
context.write(<span class="keyword">new</span> ImmutableBytesWritable(Bytes.toBytes(key.toString())), put);
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>To test this code, the first step is to add a dependency to MRUnit to your Maven POM file.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.mrunit<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>mrunit<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>1.0.0 <span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;scope&gt;</span>test<span class="tag">&lt;/scope&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Next, use the ReducerDriver provided by MRUnit, in your Reducer job.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">MyReducerTest</span> {
ReduceDriver&lt;Text, Text, ImmutableBytesWritable, Writable&gt; reduceDriver;
<span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="type">byte</span><span class="type">[]</span> QUALIFIER = <span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="annotation">@Before</span>
<span class="directive">public</span> <span class="type">void</span> setUp() {
MyReducer reducer = <span class="keyword">new</span> MyReducer();
reduceDriver = ReduceDriver.newReduceDriver(reducer);
}
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testHBaseInsert() <span class="directive">throws</span> <span class="exception">IOException</span> {
<span class="predefined-type">String</span> strKey = <span class="string"><span class="delimiter">&quot;</span><span class="content">RowKey-1</span><span class="delimiter">&quot;</span></span>, strValue = <span class="string"><span class="delimiter">&quot;</span><span class="content">DATA</span><span class="delimiter">&quot;</span></span>, strValue1 = <span class="string"><span class="delimiter">&quot;</span><span class="content">DATA1</span><span class="delimiter">&quot;</span></span>,
strValue2 = <span class="string"><span class="delimiter">&quot;</span><span class="content">DATA2</span><span class="delimiter">&quot;</span></span>;
<span class="predefined-type">List</span>&lt;Text&gt; list = <span class="keyword">new</span> <span class="predefined-type">ArrayList</span>&lt;Text&gt;();
list.add(<span class="keyword">new</span> Text(strValue));
list.add(<span class="keyword">new</span> Text(strValue1));
list.add(<span class="keyword">new</span> Text(strValue2));
<span class="comment">//since in our case all that the reducer is doing is appending the records that the mapper</span>
<span class="comment">//sends it, we should get the following back</span>
<span class="predefined-type">String</span> expectedOutput = strValue + strValue1 + strValue2;
<span class="comment">//Setup Input, mimic what mapper would have passed</span>
<span class="comment">//to the reducer and run test</span>
reduceDriver.withInput(<span class="keyword">new</span> Text(strKey), list);
<span class="comment">//run the reducer and get its output</span>
<span class="predefined-type">List</span>&lt;Pair&lt;ImmutableBytesWritable, Writable&gt;&gt; result = reduceDriver.run();
<span class="comment">//extract key from result and verify</span>
assertEquals(Bytes.toString(result.get(<span class="integer">0</span>).getFirst().get()), strKey);
<span class="comment">//extract value for CF/QUALIFIER and verify</span>
Put a = (Put)result.get(<span class="integer">0</span>).getSecond();
<span class="predefined-type">String</span> c = Bytes.toString(a.get(CF, QUALIFIER).get(<span class="integer">0</span>).getValue());
assertEquals(expectedOutput,c );
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>Your MRUnit test verifies that the output is as expected, the Put that is inserted into HBase has the correct value, and the ColumnFamily and ColumnQualifier have the correct values.</p>
</div>
<div class="paragraph">
<p>MRUnit includes a MapperDriver to test mapping jobs, and you can use MRUnit to test other operations, including reading from HBase, processing data, or writing to HDFS,</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_integration_testing_with_an_hbase_mini_cluster"><a class="anchor" href="#_integration_testing_with_an_hbase_mini_cluster"></a>187. Integration Testing with an HBase Mini-Cluster</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase ships with HBaseTestingUtility, which makes it easy to write integration tests using a <em class="firstterm">mini-cluster</em>.
The first step is to add some dependencies to your Maven POM file.
Check the versions to be sure they are appropriate.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;properties&gt;</span>
<span class="tag">&lt;hbase.version&gt;</span>2.0.0-SNAPSHOT<span class="tag">&lt;/hbase.version&gt;</span>
<span class="tag">&lt;/properties&gt;</span>
<span class="tag">&lt;dependencies&gt;</span>
<span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.hbase<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>hbase-testing-util<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>${hbase.version}<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;scope&gt;</span>test<span class="tag">&lt;/scope&gt;</span>
<span class="tag">&lt;/dependency&gt;</span>
<span class="tag">&lt;/dependencies&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>This code represents an integration test for the MyDAO insert shown in <a href="#unit.tests">unit.tests</a>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">class</span> <span class="class">MyHBaseIntegrationTest</span> {
<span class="directive">private</span> <span class="directive">static</span> HBaseTestingUtility utility;
<span class="type">byte</span><span class="type">[]</span> CF = <span class="string"><span class="delimiter">&quot;</span><span class="content">CF</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="type">byte</span><span class="type">[]</span> CQ1 = <span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-1</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="type">byte</span><span class="type">[]</span> CQ2 = <span class="string"><span class="delimiter">&quot;</span><span class="content">CQ-2</span><span class="delimiter">&quot;</span></span>.getBytes();
<span class="annotation">@Before</span>
<span class="directive">public</span> <span class="type">void</span> setup() <span class="directive">throws</span> <span class="exception">Exception</span> {
utility = <span class="keyword">new</span> HBaseTestingUtility();
utility.startMiniCluster();
}
<span class="annotation">@Test</span>
<span class="directive">public</span> <span class="type">void</span> testInsert() <span class="directive">throws</span> <span class="exception">Exception</span> {
Table table = utility.createTable(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">MyTest</span><span class="delimiter">&quot;</span></span>), CF);
HBaseTestObj obj = <span class="keyword">new</span> HBaseTestObj();
obj.setRowKey(<span class="string"><span class="delimiter">&quot;</span><span class="content">ROWKEY-1</span><span class="delimiter">&quot;</span></span>);
obj.setData1(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-1</span><span class="delimiter">&quot;</span></span>);
obj.setData2(<span class="string"><span class="delimiter">&quot;</span><span class="content">DATA-2</span><span class="delimiter">&quot;</span></span>);
MyHBaseDAO.insertRecord(table, obj);
Get get1 = <span class="keyword">new</span> Get(Bytes.toBytes(obj.getRowKey()));
get1.addColumn(CF, CQ1);
<span class="predefined-type">Result</span> result1 = table.get(get1);
assertEquals(Bytes.toString(result1.getRow()), obj.getRowKey());
assertEquals(Bytes.toString(result1.value()), obj.getData1());
Get get2 = <span class="keyword">new</span> Get(Bytes.toBytes(obj.getRowKey()));
get2.addColumn(CF, CQ2);
<span class="predefined-type">Result</span> result2 = table.get(get2);
assertEquals(Bytes.toString(result2.getRow()), obj.getRowKey());
assertEquals(Bytes.toString(result2.value()), obj.getData2());
}
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>This code creates an HBase mini-cluster and starts it.
Next, it creates a table called <code>MyTest</code> with one column family, <code>CF</code>.
A record is inserted, a Get is performed from the same table, and the insertion is verified.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Starting the mini-cluster takes about 20-30 seconds, but that should be appropriate for integration testing.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>See the paper at <a href="http://blog.sematext.com/2010/08/30/hbase-case-study-using-hbasetestingutility-for-local-testing-development/">HBase Case-Study: Using HBaseTestingUtility for Local Testing and
Development</a> (2010) for more information about HBaseTestingUtility.</p>
</div>
</div>
</div>
<h1 id="protobuf" class="sect0"><a class="anchor" href="#protobuf"></a>Protobuf in HBase</h1>
<div class="sect1">
<h2 id="_protobuf"><a class="anchor" href="#_protobuf"></a>188. Protobuf</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase uses Google&#8217;s <a href="https://developers.google.com/protocol-buffers/">protobufs</a> wherever
it persists metadata&#8201;&#8212;&#8201;in the tail of hfiles or Cells written by
HBase into the system hbase:meta table or when HBase writes znodes
to zookeeper, etc.&#8201;&#8212;&#8201;and when it passes objects over the wire making
<a href="#hbase.rpc">RPCs</a>. HBase uses protobufs to describe the RPC
Interfaces (Services) we expose to clients, for example the <code>Admin</code> and <code>Client</code>
Interfaces that the RegionServer fields,
or specifying the arbitrary extensions added by developers via our
<a href="#cp">Coprocessor Endpoint</a> mechanism.</p>
</div>
<div class="paragraph">
<p>In this chapter we go into detail for developers who are looking to
understand better how it all works. This chapter is of particular
use to those who would amend or extend HBase functionality.</p>
</div>
<div class="paragraph">
<p>With protobuf, you describe serializations and services in a <code>.protos</code> file.
You then feed these descriptors to a protobuf tool, the <code>protoc</code> binary,
to generate classes that can marshall and unmarshall the described serializations
and field the specified Services.</p>
</div>
<div class="paragraph">
<p>See the <code>README.txt</code> in the HBase sub-modules for details on how
to run the class generation on a per-module basis;
e.g. see <code>hbase-protocol/README.txt</code> for how to generate protobuf classes
in the hbase-protocol module.</p>
</div>
<div class="paragraph">
<p>In HBase, <code>.proto</code> files are either in the <code>hbase-protocol</code> module; a module
dedicated to hosting the common proto files and the protoc generated classes
that HBase uses internally serializing metadata. For extensions to hbase
such as REST or Coprocessor Endpoints that need their own descriptors; their
protos are located inside the function&#8217;s hosting module: e.g. <code>hbase-rest</code>
is home to the REST proto files and the <code>hbase-rsgroup</code> table grouping
Coprocessor Endpoint has all protos that have to do with table grouping.</p>
</div>
<div class="paragraph">
<p>Protos are hosted by the module that makes use of them. While
this makes it so generation of protobuf classes is distributed, done
per module, we do it this way so modules encapsulate all to do with
the functionality they bring to hbase.</p>
</div>
<div class="paragraph">
<p>Extensions whether REST or Coprocessor Endpoints will make use
of core HBase protos found back in the hbase-protocol module. They&#8217;ll
use these core protos when they want to serialize a Cell or a Put or
refer to a particular node via ServerName, etc., as part of providing the
CPEP Service. Going forward, after the release of hbase-2.0.0, this
practice needs to whither. We&#8217;ll explain why in the later
<a href="#shaded.protobuf">hbase-2.0.0</a> section.</p>
</div>
<div class="sect2">
<h3 id="shaded.protobuf"><a class="anchor" href="#shaded.protobuf"></a>188.1. hbase-2.0.0 and the shading of protobufs (HBASE-15638)</h3>
<div class="paragraph">
<p>As of hbase-2.0.0, our protobuf usage gets a little more involved. HBase
core protobuf references are offset so as to refer to a private,
bundled protobuf. Core stops referring to protobuf
classes at com.google.protobuf.* and instead references protobuf at
the HBase-specific offset
org.apache.hadoop.hbase.shaded.com.google.protobuf.*. We do this indirection
so hbase core can evolve its protobuf version independent of whatever our
dependencies rely on. For instance, HDFS serializes using protobuf.
HDFS is on our CLASSPATH. Without the above described indirection, our
protobuf versions would have to align. HBase would be stuck
on the HDFS protobuf version until HDFS decided to upgrade. HBase
and HDFS versions would be tied.</p>
</div>
<div class="paragraph">
<p>We had to move on from protobuf-2.5.0 because we need facilities
added in protobuf-3.1.0; in particular being able to save on
copies and avoiding bringing protobufs onheap for
serialization/deserialization.</p>
</div>
<div class="paragraph">
<p>In hbase-2.0.0, we introduced a new module, <code>hbase-protocol-shaded</code>
inside which we contained all to do with protobuf and its subsequent
relocation/shading. This module is in essence a copy of much of the old
<code>hbase-protocol</code> but with an extra shading/relocation step.
Core was moved to depend on this new module.</p>
</div>
<div class="paragraph">
<p>That said, a complication arises around Coprocessor Endpoints (CPEPs).
CPEPs depend on public HBase APIs that reference protobuf classes at
<code>com.google.protobuf.*</code> explicitly. For example, in our Table Interface
we have the below as the means by which you obtain a CPEP Service
to make invocations against:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
&lt;T <span class="directive">extends</span> com.google.protobuf.Service,R&gt; <span class="predefined-type">Map</span>&lt;<span class="type">byte</span><span class="type">[]</span>,R&gt; coprocessorService(
<span class="predefined-type">Class</span>&lt;T&gt; service, <span class="type">byte</span><span class="type">[]</span> startKey, <span class="type">byte</span><span class="type">[]</span> endKey,
org.apache.hadoop.hbase.client.coprocessor.Batch.Call&lt;T,R&gt; callable)
<span class="directive">throws</span> com.google.protobuf.ServiceException, <span class="predefined-type">Throwable</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Existing CPEPs will have made reference to core HBase protobufs
specifying ServerNames or carrying Mutations.
So as to continue being able to service CPEPs and their references
to <code>com.google.protobuf.<strong></code> across the upgrade to hbase-2.0.0 and beyond,
HBase needs to be able to deal with both
<code>com.google.protobuf.</strong></code> references and its internal offset
<code>org.apache.hadoop.hbase.shaded.com.google.protobuf.*</code> protobufs.</p>
</div>
<div class="paragraph">
<p>The <code>hbase-protocol-shaded</code> module hosts all
protobufs used by HBase core.</p>
</div>
<div class="paragraph">
<p>But for the vestigial CPEP references to the (non-shaded) content of
<code>hbase-protocol</code>, we keep around most of this module going forward
just so it is available to CPEPs. Retaining the most of <code>hbase-protocol</code>
makes for overlapping, 'duplicated' proto instances where some exist as
non-shaded/non-relocated here in their old module
location but also in the new location, shaded under
<code>hbase-protocol-shaded</code>. In other words, there is an instance
of the generated protobuf class
<code>org.apache.hadoop.hbase.protobuf.generated.ServerName</code>
in hbase-protocol and another generated instance that is the same in all
regards except its protobuf references are to the internal shaded
version at <code>org.apache.hadoop.hbase.shaded.protobuf.generated.ServerName</code>
(note the 'shaded' addition in the middle of the package name).</p>
</div>
<div class="paragraph">
<p>If you extend a proto in <code>hbase-protocol-shaded</code> for internal use,
consider extending it also in
<code>hbase-protocol</code> (and regenerating).</p>
</div>
<div class="paragraph">
<p>Going forward, we will provide a new module of common types for use
by CPEPs that will have the same guarantees against change as does our
public API. TODO.</p>
</div>
</div>
</div>
</div>
<h1 id="pv2" class="sect0"><a class="anchor" href="#pv2"></a>Procedure Framework (Pv2): <a href="https://issues.apache.org/jira/browse/HBASE-12439">HBASE-12439</a></h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p><em>Procedure v2 &#8230;&#8203;aims to provide a unified way to build&#8230;&#8203;multi-step procedures with a rollback/roll-forward ability in case of failure (e.g. create/delete table)&#8201;&#8212;&#8201;Matteo Bertozzi, the author of Pv2.</em></p>
</div>
<div class="paragraph">
<p>With Pv2 you can build and run state machines. It was built by Matteo to make distributed state transitions in HBase resilient in the face of process failures. Previous to Pv2, state transition handling was spread about the codebase with implementation varying by transition-type and context. Pv2 was inspired by <a href="https://accumulo.apache.org/1.8/accumulo_user_manual.html#_fault_tolerant_executor_fate">FATE</a>, of Apache Accumulo.<br></p>
</div>
<div class="paragraph">
<p>Early Pv2 aspects have been shipping in HBase with a good while now but it has continued to evolve as it takes on more involved scenarios. What we have now is powerful but intricate in operation and incomplete, in need of cleanup and hardening. In this doc we have given overview on the system so you can make use of it (and help with its polishing).</p>
</div>
<div class="paragraph">
<p>This system has the awkward name of Pv2 because HBase already had the notion of a Procedure used in snapshots (see hbase-server <em>org.apache.hadoop.hbase.procedure</em> as opposed to hbase-procedure <em>org.apache.hadoop.hbase.procedure2</em>). Pv2 supercedes and is to replace Procedure.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_procedures"><a class="anchor" href="#_procedures"></a>189. Procedures</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A Procedure is a transform made on an HBase entity. Examples of HBase entities would be Regions and Tables.<br>
Procedures are run by a ProcedureExecutor instance. Procedure current state is kept in the ProcedureStore.<br>
The ProcedureExecutor has but a primitive view on what goes on inside a Procedure. From its PoV, Procedures are submitted and then the ProcedureExecutor keeps calling <em>#execute(Object)</em> until the Procedure is done. Execute may be called multiple times in the case of failure or restart, so Procedure code must be idempotent yielding the same result each time it run. Procedure code can also implement <em>rollback</em> so steps can be undone if failure. A call to <em>execute()</em> can result in one of following possibilities:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>execute()</em> returns</p>
<div class="ulist">
<ul>
<li>
<p><em>null</em>: indicates we are done.</p>
</li>
<li>
<p><em>this</em>: indicates there is more to do so, persist current procedure state and re-<em>execute()</em>.</p>
</li>
<li>
<p><em>Array</em> of sub-procedures: indicates a set of procedures needed to be run to completion before we can proceed (after which we expect the framework to call our execute again).</p>
</li>
</ul>
</div>
</li>
<li>
<p><em>execute()</em> throws exception</p>
<div class="ulist">
<ul>
<li>
<p><em>suspend</em>: indicates execution of procedure is suspended and can be resumed due to some external event. The procedure state is persisted.</p>
</li>
<li>
<p><em>yield</em>: procedure is added back to scheduler. The procedure state is not persisted.</p>
</li>
<li>
<p><em>interrupted</em>: currently same as <em>yield</em>.</p>
</li>
<li>
<p>Any <em>exception</em> not listed above: Procedure <em>state</em> is changed to <em>FAILED</em> (after which we expect the framework will attempt rollback).</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
<div class="paragraph">
<p>The ProcedureExecutor stamps the frameworks notions of Procedure State into the Procedure itself; e.g. it marks Procedures as INITIALIZING on submit. It moves the state to RUNNABLE when it goes to execute. When done, a Procedure gets marked FAILED or SUCCESS depending. Here is the list of all states as of this writing:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><strong><em>INITIALIZING</em></strong> Procedure in construction, not yet added to the executor</p>
</li>
<li>
<p><strong><em>RUNNABLE</em></strong> Procedure added to the executor, and ready to be executed.</p>
</li>
<li>
<p><strong><em>WAITING</em></strong> The procedure is waiting on children (subprocedures) to be completed</p>
</li>
<li>
<p><strong><em>WAITING_TIMEOUT</em></strong> The procedure is waiting a timeout or an external event</p>
</li>
<li>
<p><strong><em>ROLLEDBACK</em></strong> The procedure failed and was rolledback.</p>
</li>
<li>
<p><strong><em>SUCCESS</em></strong> The procedure execution completed successfully.</p>
</li>
<li>
<p><strong><em>FAILED</em></strong> The procedure execution failed, may need to rollback.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>After each execute, the Procedure state is persisted to the ProcedureStore. Hooks are invoked on Procedures so they can preserve custom state. Post-fault, the ProcedureExecutor re-hydrates its pre-crash state by replaying the content of the ProcedureStore. This makes the Procedure Framework resilient against process failure.</p>
</div>
<div class="sect2">
<h3 id="_implementation"><a class="anchor" href="#_implementation"></a>189.1. Implementation</h3>
<div class="paragraph">
<p>In implementation, Procedures tend to divide transforms into finer-grained tasks and while some of these work items are handed off to sub-procedures,
the bulk are done as processing <em>steps</em> in-Procedure; each invocation of the execute is used to perform a single step, and then the Procedure relinquishes returning to the framework. The Procedure does its own tracking of where it is in the processing.</p>
</div>
<div class="paragraph">
<p>What comprises a sub-task, or <em>step</em> in the execution is up to the Procedure author but generally it is a small piece of work that cannot be further decomposed and that moves the processing forward toward its end state. Having procedures made of many small steps rather than a few large ones allows the Procedure framework give out insight on where we are in the processing. It also allows the framework be more fair in its execution. As stated per above, each step may be called multiple times (failure/restart) so steps must be implemented idempotent.<br>
It is easy to confuse the state that the Procedure itself is keeping with that of the Framework itself. Try to keep them distinct.<br></p>
</div>
</div>
<div class="sect2">
<h3 id="_rollback_2"><a class="anchor" href="#_rollback_2"></a>189.2. Rollback</h3>
<div class="paragraph">
<p>Rollback is called when the procedure or one of the sub-procedures has failed. The rollback step is supposed to cleanup the resources created during the execute() step. In case of failure and restart, rollback() may be called multiple times, so again the code must be idempotent.</p>
</div>
</div>
<div class="sect2">
<h3 id="_metrics_2"><a class="anchor" href="#_metrics_2"></a>189.3. Metrics</h3>
<div class="paragraph">
<p>There are hooks for collecting metrics on submit of the procedure and on finish.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>updateMetricsOnSubmit()</p>
</li>
<li>
<p>updateMetricsOnFinish()</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Individual procedures can override these methods to collect procedure specific metrics. The default implementations of these methods try to get an object implementing an interface ProcedureMetrics which encapsulates following set of generic metrics:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>SubmittedCount (Counter): Total number of procedure instances submitted of a type.</p>
</li>
<li>
<p>Time (Histogram): Histogram of runtime for procedure instances.</p>
</li>
<li>
<p>FailedCount (Counter): Total number of failed procedure instances.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Individual procedures can implement this object and define these generic set of metrics.</p>
</div>
</div>
<div class="sect2">
<h3 id="_baggage"><a class="anchor" href="#_baggage"></a>189.4. Baggage</h3>
<div class="paragraph">
<p>Procedures can carry baggage. One example is the <em>step</em> the procedure last attained (see previous section); procedures persist the enum that marks where they are currently. Other examples might be the Region or Server name the Procedure is currently working. After each call to execute, the Procedure#serializeStateData is called. Procedures can persist whatever.</p>
</div>
</div>
<div class="sect2">
<h3 id="_result_state_and_queries"><a class="anchor" href="#_result_state_and_queries"></a>189.5. Result/State and Queries</h3>
<div class="paragraph">
<p>(From Matteo’s <a href="https://issues.apache.org/jira/secure/attachment/12693273/Procedurev2Notification-Bus.pdf">ProcedureV2 and Notification Bus</a> doc)<br>
In the case of asynchronous operations, the result must be kept around until the client asks for it. Once we receive a “get” of the result we can schedule the delete of the record. For some operations the result may be “unnecessary” especially in case of failure (e.g. if the create table fail, we can query the operation result or we can just do a list table to see if it was created) so in some cases we can schedule the delete after a timeout. On the client side the operation will return a “Procedure ID”, this ID can be used to wait until the procedure is completed and get the result/exception.<br></p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Admin.doOperation() { longprocId=master.doOperation(); master.waitCompletion(procId); } +</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the master goes down while performing the operation the backup master will pickup the half in­progress operation and complete it. The client will not notice the failure.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_subprocedures"><a class="anchor" href="#_subprocedures"></a>190. Subprocedures</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Subprocedures are <em>Procedure</em> instances created and returned by <em>#execute(Object)</em> method of a procedure instance (parent procedure). As subprocedures are of type <em>Procedure</em>, they can instantiate their own subprocedures. As its a recursive, procedure stack is maintained by the framework. The framework makes sure that the parent procedure does not proceed till all sub-procedures and their subprocedures in a procedure stack are successfully finished.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_procedureexecutor"><a class="anchor" href="#_procedureexecutor"></a>191. ProcedureExecutor</h2>
<div class="sectionbody">
<div class="paragraph">
<p><em>ProcedureExecutor</em> uses <em>ProcedureStore</em> and <em>ProcedureScheduler</em> and executes procedures submitted to it. Some of the basic operations supported are:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><em>abort(procId)</em>: aborts specified procedure if its not finished</p>
</li>
<li>
<p><em>submit(Procedure)</em>: submits procedure for execution</p>
</li>
<li>
<p><em>retrieve:</em> list of get methods to get <em>Procedure</em> instances and results</p>
</li>
<li>
<p><em>register/ unregister</em> listeners: for listening on Procedure related notifications</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>When <em>ProcedureExecutor</em> starts it loads procedure instances persisted in <em>ProcedureStore</em> from previous run. All unfinished procedures are resumed from the last stored state.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_nonces"><a class="anchor" href="#_nonces"></a>192. Nonces</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can pass the nonce that came in with the RPC to the Procedure on submit at the executor. This nonce will then be serialized along w/ the Procedure on persist. If a crash, on reload, the nonce will be put back into a map of nonces to pid in case a client tries to run same procedure for a second time (it will be rejected). See the base Procedure and how nonce is a base data member.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_wait_wake_suspend_yield"><a class="anchor" href="#_wait_wake_suspend_yield"></a>193. Wait/Wake/Suspend/Yield</h2>
<div class="sectionbody">
<div class="paragraph">
<p>‘suspend’ means stop processing a procedure because we can make no more progress until a condition changes; i.e. we sent RPC and need to wait on response. The way this works is that a Procedure throws a suspend exception from down in its guts as a GOTO the end-of-the-current-processing step. Suspend also puts the Procedure back on the scheduler. Problematic is we do some accounting on our way out even on suspend making it so it can take time exiting (We have to update state in the WAL).</p>
</div>
<div class="paragraph">
<p>RegionTransitionProcedure#reportTransition is called on receipt of a report from a RS. For Assign and Unassign, this event response from the server we sent an RPC wakes up suspended Assign/Unassigns.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_locking"><a class="anchor" href="#_locking"></a>194. Locking</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Procedure Locks are not about concurrency! They are about giving a Procedure read/write access to an HBase Entity such as a Table or Region so that is possible to shut out other Procedures from making modifications to an HBase Entity state while the current one is running.</p>
</div>
<div class="paragraph">
<p>Locking is optional, up to the Procedure implementor but if an entity is being operated on by a Procedure, all transforms need to be done via Procedures using the same locking scheme else havoc.</p>
</div>
<div class="paragraph">
<p>Two ProcedureExecutor Worker threads can actually end up both processing the same Procedure instance. If it happens, the threads are meant to be running different parts of the one Procedure&#8201;&#8212;&#8201;changes that do not stamp on each other (This gets awkward around the procedure frameworks notion of ‘suspend’. More on this below).</p>
</div>
<div class="paragraph">
<p>Locks optionally may be held for the life of a Procedure. For example, if moving a Region, you probably want to have exclusive access to the HBase Region until the Region completes (or fails). This is used in conjunction with {@link #holdLock(Object)}. If {@link #holdLock(Object)} returns true, the procedure executor will call acquireLock() once and thereafter not call {@link #releaseLock(Object)} until the Procedure is done (Normally, it calls release/acquire around each invocation of {@link #execute(Object)}.</p>
</div>
<div class="paragraph">
<p>Locks also may live the life of a procedure; i.e. once an Assign Procedure starts, we do not want another procedure meddling w/ the region under assignment. Procedures that hold the lock for the life of the procedure set Procedure#holdLock to true. AssignProcedure does this as do Split and Move (If in the middle of a Region move, you do not want it Splitting).</p>
</div>
<div class="paragraph">
<p>Locking can be for life of Procedure.</p>
</div>
<div class="paragraph">
<p>Some locks have a hierarchy. For example, taking a region lock also takes (read) lock on its containing table and namespace to prevent another Procedure obtaining an exclusive lock on the hosting table (or namespace).</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_procedure_types"><a class="anchor" href="#_procedure_types"></a>195. Procedure Types</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_statemachineprocedure"><a class="anchor" href="#_statemachineprocedure"></a>195.1. StateMachineProcedure</h3>
<div class="paragraph">
<p>One can consider each call to <em>#execute(Object)</em> method as transitioning from one state to another in a state machine. Abstract class <em>StateMachineProcedure</em> is wrapper around base <em>Procedure</em> class which provides constructs for implementing a state machine as a <em>Procedure</em>. After each state transition current state is persisted so that, in case of crash/ restart, the state transition can be resumed from the previous state of a procedure before crash/ restart. Individual procedures need to define initial and terminus states and hooks <em>executeFromState()</em> and <em>setNextState()</em> are provided for state transitions.</p>
</div>
</div>
<div class="sect2">
<h3 id="_remoteproceduredispatcher"><a class="anchor" href="#_remoteproceduredispatcher"></a>195.2. RemoteProcedureDispatcher</h3>
<div class="paragraph">
<p>A new RemoteProcedureDispatcher (+ subclass RSProcedureDispatcher) primitive takes care of running the Procedure-based Assignments ‘remote’ component. This dispatcher knows about ‘servers’. It does aggregation of assignments by time on a time/count basis so can send procedures in batches rather than one per RPC. Procedure status comes back on the back of the RegionServer heartbeat reporting online/offline regions (No more notifications via ZK). The response is passed to the AMv2 to ‘process’. It will check against the in-memory state. If there is a mismatch, it fences out the RegionServer on the assumption that something went wrong on the RS side. Timeouts trigger retries (Not Yet Implemented!). The Procedure machine ensures only one operation at a time on any one Region/Table using entity <em>locking</em> and smarts about what is serial and what can be run concurrently (Locking was zk-based&#8201;&#8212;&#8201;you’d put a znode in zk for a table&#8201;&#8212;&#8201;but now has been converted to be procedure-based as part of this project).</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_references"><a class="anchor" href="#_references"></a>196. References</h2>
<div class="sectionbody">
<div class="ulist">
<ul>
<li>
<p>Matteo had a slide deck on what it the Procedure Framework would look like and the problems it addresses initially <a href="https://issues.apache.org/jira/secure/attachment/12845124/ProcedureV2b.pdf">attached to the Pv2 issue.</a></p>
</li>
<li>
<p><a href="https://issues.apache.org/jira/secure/attachment/12693273/Procedurev2Notification-Bus.pdf">A good doc by Matteo</a> on problem and how Pv2 addresses it w/ roadmap (from the Pv2 JIRA). We should go back to the roadmap to do the Notification Bus, convertion of log splitting to Pv2, etc.</p>
</li>
</ul>
</div>
</div>
</div>
<h1 id="amv2" class="sect0"><a class="anchor" href="#amv2"></a>AMv2 Description for Devs</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>The AssignmentManager (AM) in HBase Master manages assignment of Regions over a cluster of RegionServers.</p>
</div>
<div class="paragraph">
<p>The AMv2 project is a redo of Assignment in an attempt at addressing the root cause of many of our operational issues in production, namely slow assignment and problematic accounting such that Regions are misplaced stuck offline in the notorious <em>Regions-In-Transition (RIT)</em> limbo state.</p>
</div>
<div class="paragraph">
<p>Below are notes for devs on key aspects of AMv2 in no particular order.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_background_3"><a class="anchor" href="#_background_3"></a>197. Background</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Assignment in HBase 1.x has been problematic in operation. It is not hard to see why. Region state is kept at the other end of an RPC in ZooKeeper (Terminal states&#8201;&#8212;&#8201;i.e. OPEN or CLOSED&#8201;&#8212;&#8201;are published to the <em>hbase:meta</em> table). In HBase-1.x.x, state has multiple writers with Master and RegionServers all able to make state edits concurrently (in <em>hbase:meta</em> table and out on ZooKeeper). If clocks are awry or watchers missed, state changes can be skipped or overwritten. Locking of HBase Entities&#8201;&#8212;&#8201;tables, regions&#8201;&#8212;&#8201;is not comprehensive so a table operation&#8201;&#8212;&#8201;disable/enable&#8201;&#8212;&#8201;could clash with a region-level operation; a split or merge. Region state is distributed and hard to reason about and test. Assignment is slow in operation because each assign involves moving remote znodes through transitions. Cluster size tends to top out at a couple of hundred thousand regions; beyond this, cluster start/stop takes hours and is prone to corruption.</p>
</div>
<div class="paragraph">
<p>AMv2 (AssignmentManager Version 2) is a refactor (<a href="https://issues.apache.org/jira/browse/HBASE-14350">HBASE-14350</a>) of the hbase-1.x AssignmentManager putting it up on a <a href="https://issues.apache.org/jira/browse/HBASE-12439">ProcedureV2 (HBASE-12439)</a> basis. ProcedureV2 (Pv2)<em>,</em> is an awkwardly named system that allows describing and running multi-step state machines. It is performant and persists all state to a Store which is recoverable post crash. See the companion chapter on <a href="#pv2">Procedure Framework (Pv2): <a href="https://issues.apache.org/jira/browse/HBASE-12439">HBASE-12439</a></a>, to learn more about the ProcedureV2 system.</p>
</div>
<div class="paragraph">
<p>In AMv2, all assignment, crash handling, splits and merges are recast as Procedures(v2). ZooKeeper is purged from the mix. As before, the final assignment state gets published to <em>hbase:meta</em> for non-Master participants to read (all-clients) with intermediate state kept in the local Pv2 WAL-based ‘store’ but only the active Master, a single-writer, evolves state. The Master’s in-memory cluster image is the authority and if disagreement, RegionServers are forced to comply. Pv2 adds shared/exclusive locking of all core HBase Entities&#8201;&#8212;&#8201;namespace, tables, and regions&#8201;&#8212;&#8201;to ensure one actor at a time access and to prevent operations contending over resources (move/split, disable/assign, etc.).</p>
</div>
<div class="paragraph">
<p>This redo of AM atop of a purposed, performant state machine with all operations taking on the common Procedure form with a single state writer only moves our AM to a new level of resilience and scale.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_new_system"><a class="anchor" href="#_new_system"></a>198. New System</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Each Region Assign or Unassign of a Region is now a Procedure. A Move (Region) Procedure is a compound of Procedures; it is the running of an Unassign Procedure followed by an Assign Procedure. The Move Procedure spawns the Assign and Unassign in series and then waits on their completions.</p>
</div>
<div class="paragraph">
<p>And so on. ServerCrashProcedure spawns the WAL splitting tasks and then the reassign of all regions that were hosted on the crashed server as subprocedures.</p>
</div>
<div class="paragraph">
<p>AMv2 Procedures are run by the Master in a ProcedureExecutor instance. All Procedures make use of utility provided by the Pv2 framework.</p>
</div>
<div class="paragraph">
<p>For example, Procedures persist each state transition to the frameworks’ Procedure Store. The default implementation is done as a WAL kept on HDFS. On crash, we reopen the Store and rerun all WALs of Procedure transitions to put the Assignment State Machine back into the attitude it had just before crash. We then continue Procedure execution.</p>
</div>
<div class="paragraph">
<p>In the new system, the Master is the Authority on all things Assign. Previous we were ambiguous; e.g. the RegionServer was in charge of Split operations. Master keeps an in-memory image of Region states and servers. If disagreement, the Master always prevails; at an extreme it will kill the RegionServer that is in disagreement.</p>
</div>
<div class="paragraph">
<p>A new RegionStateStore class takes care of publishing the terminal Region state, whether OPEN or CLOSED, out to the _hbase:meta _table<em>.</em></p>
</div>
<div class="paragraph">
<p>RegionServers now report their run version on Connection. This version is available inside the AM for use running migrating rolling restarts.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_procedures_detail"><a class="anchor" href="#_procedures_detail"></a>199. Procedures Detail</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_assign_unassign"><a class="anchor" href="#_assign_unassign"></a>199.1. Assign/Unassign</h3>
<div class="paragraph">
<p>Assign and Unassign subclass a common RegionTransitionProcedure. There can only be one RegionTransitionProcedure per region running at a time since the RTP instance takes a lock on the region. The RTP base Procedure has three steps; a store the procedure step (REGION_TRANSITION_QUEUE); a dispatch of the procedure open or close followed by a suspend waiting on the remote regionserver to report successful open or fail (REGION_TRANSITION_DISPATCH) or notification that the server fielding the request crashed; and finally registration of the successful open/close in hbase:meta (REGION_TRANSITION_FINISH).</p>
</div>
<div class="paragraph">
<p>Here is how the assign of a region 56f985a727afe80a184dac75fbf6860c looks in the logs. The assign was provoked by a Server Crash (Process ID 1176 or pid=1176 which when it is the parent of a procedure, it is identified as ppid=1176). The assign is pid=1179, the second region of the two being assigned by this Server Crash.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">24</span>,<span class="integer">175</span> INFO [ProcExecWrkr-<span class="integer">30</span>] procedure2.ProcedureExecutor: Initialized subprocedures=[{pid=<span class="integer">1178</span>, ppid=<span class="integer">1176</span>, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=bfd57f0b72fd3ca77e9d3c5e3ae48d76, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>}, {pid=<span class="integer">1179</span>, ppid=<span class="integer">1176</span>, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=<span class="float">56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>}]</code></pre>
</div>
</div>
<div class="paragraph">
<p>Next we start the assign by queuing (‘registering’) the Procedure with the framework.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">24</span>,<span class="integer">241</span> INFO [ProcExecWrkr-<span class="integer">30</span>] assignment.AssignProcedure: Start pid=<span class="integer">1179</span>, ppid=<span class="integer">1176</span>, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=<span class="float">56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>; rit=OFFLINE, location=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>; forceNewPlan=<span class="predefined-constant">false</span>, retain=<span class="predefined-constant">false</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Track the running of Procedures in logs by tracing their process id&#8201;&#8212;&#8201;here pid=1179.</p>
</div>
<div class="paragraph">
<p>Next we move to the dispatch phase where we update hbase:meta table setting the region state as OPENING on server ve540. We then dispatch an rpc to ve540 asking it to open the region. Thereafter we suspend the Assign until we get a message back from ve540 on whether it has opened the region successfully (or not).</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">24</span>,<span class="integer">494</span> INFO [ProcExecWrkr-<span class="integer">38</span>] assignment.RegionStateStore: pid=<span class="integer">1179</span> updating hbase:meta row=IntegrationTestBigLinkedList,H<span class="error">\</span>xE3<span class="error">@</span><span class="error">\</span>x8D<span class="error">\</span>x964<span class="error">\</span>x9D<span class="error">\</span>xDF<span class="error">\</span>x8F<span class="error">@</span><span class="integer">9</span><span class="error">\</span>x0F<span class="error">\</span>xC8<span class="error">\</span>xCC<span class="error">\</span>xC2,<span class="float">1495566261066.56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c., regionState=OPENING, regionLocation=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>
<span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">24</span>,<span class="integer">498</span> INFO [ProcExecWrkr-<span class="integer">38</span>] assignment.RegionTransitionProcedure: Dispatch pid=<span class="integer">1179</span>, ppid=<span class="integer">1176</span>, state=RUNNABLE:REGION_TRANSITION_DISPATCH; AssignProcedure table=IntegrationTestBigLinkedList, region=<span class="float">56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>; rit=OPENING, location=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Below we log the incoming report that the region opened successfully on ve540. The Procedure is woken up (you can tell it the procedure is running by the name of the thread, its a ProcedureExecutor thread, ProcExecWrkr-9). The woken up Procedure updates state in hbase:meta to denote the region as open on ve0540. It then reports finished and exits.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">26</span>,<span class="integer">643</span> DEBUG [RpcServer.default.FPBQ.Fifo.handler=<span class="integer">46</span>,queue=<span class="integer">1</span>,port=<span class="integer">16000</span>] assignment.RegionTransitionProcedure: Received report OPENED seqId=<span class="integer">11984985</span>, pid=<span class="integer">1179</span>, ppid=<span class="integer">1176</span>, state=RUNNABLE:REGION_TRANSITION_DISPATCH; AssignProcedure table=IntegrationTestBigLinkedList, region=<span class="float">56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>; rit=OPENING, location=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span> <span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">26</span>,<span class="integer">643</span> INFO [ProcExecWrkr-<span class="integer">9</span>] assignment.RegionStateStore: pid=<span class="integer">1179</span> updating hbase:meta row=IntegrationTestBigLinkedList,H<span class="error">\</span>xE3<span class="error">@</span><span class="error">\</span>x8D<span class="error">\</span>x964<span class="error">\</span>x9D<span class="error">\</span>xDF<span class="error">\</span>x8F<span class="error">@</span><span class="integer">9</span><span class="error">\</span>x0F<span class="error">\</span>xC8<span class="error">\</span>xCC<span class="error">\</span>xC2,<span class="float">1495566261066.56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c., regionState=OPEN, openSeqNum=<span class="integer">11984985</span>, regionLocation=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span>
<span class="integer">2017</span>-<span class="octal">05</span>-<span class="integer">23</span> <span class="integer">12</span>:<span class="octal">04</span>:<span class="integer">26</span>,<span class="integer">836</span> INFO [ProcExecWrkr-<span class="integer">9</span>] procedure2.ProcedureExecutor: Finish suprocedure pid=<span class="integer">1179</span>, ppid=<span class="integer">1176</span>, state=SUCCESS; AssignProcedure table=IntegrationTestBigLinkedList, region=<span class="float">56f</span><span class="integer">985</span>a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,<span class="integer">16020</span>,<span class="integer">1495525111232</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Unassign looks similar given it is based on the base RegionTransitionProcedure. It has the same state transitions and does basically the same steps but with different state name (CLOSING, CLOSED).</p>
</div>
<div class="paragraph">
<p>Most other procedures are subclasses of a Pv2 StateMachine implementation. We have both Table and Region focused StateMachines types.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_ui_2"><a class="anchor" href="#_ui_2"></a>200. UI</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Along the top-bar on the Master, you can now find a ‘Procedures&amp;Locks’ tab which takes you to a page that is ugly but useful. It dumps currently running procedures and framework locks. Look at this when you can’t figure what stuff is stuck; it will at least identify problematic procedures (take the pid and grep the logs…). Look for ROLLEDBACK or pids that have been RUNNING for a long time.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_logging"><a class="anchor" href="#_logging"></a>201. Logging</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Procedures log their process ids as pid= and their parent ids (ppid=) everywhere. Work has been done so you can grep the pid and see history of a procedure operation.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_implementation_notes"><a class="anchor" href="#_implementation_notes"></a>202. Implementation Notes</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In this section we note some idiosyncrasies of operation as an attempt at saving you some head-scratching.</p>
</div>
<div class="sect2">
<h3 id="_region_transition_rpc_and_rs_heartbeat_can_arrive_at_same_time_on_master"><a class="anchor" href="#_region_transition_rpc_and_rs_heartbeat_can_arrive_at_same_time_on_master"></a>202.1. Region Transition RPC and RS Heartbeat can arrive at ~same time on Master</h3>
<div class="paragraph">
<p>Reporting Region Transition on a RegionServer is now a RPC distinct from RS heartbeating (‘RegionServerServices’ Service). An heartbeat and a status update can arrive at the Master at about the same time. The Master will update its internal state for a Region but this same state is checked when heartbeat processing. We may find the unexpected; i.e. a Region just reported as CLOSED so heartbeat is surprised to find region OPEN on the back of the RS report. In the new system, all slaves must cow to the Masters’ understanding of cluster state; the Master will kill/close any misaligned entities.</p>
</div>
<div class="paragraph">
<p>To address the above, we added a lastUpdate for in-memory Master state. Let a region state have some vintage before we act on it (one second currently).</p>
</div>
</div>
<div class="sect2">
<h3 id="_master_as_regionserver_or_as_regionserver_that_just_does_system_tables"><a class="anchor" href="#_master_as_regionserver_or_as_regionserver_that_just_does_system_tables"></a>202.2. Master as RegionServer or as RegionServer that just does system tables</h3>
<div class="paragraph">
<p>AMv2 enforces current master branch default of HMaster carrying system tables only; i.e. the Master in an HBase cluster acts also as a RegionServer only it is the exclusive host for tables such as <em>hbase:meta</em>, <em>hbase:namespace</em>, etc., the core system tables. This is causing a couple of test failures as AMv1, though it is not supposed to, allows moving hbase:meta off Master while AMv2 does not.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_new_configs"><a class="anchor" href="#_new_configs"></a>203. New Configs</h2>
<div class="sectionbody">
<div class="paragraph">
<p>These configs all need doc on when you’d change them.</p>
</div>
<div class="sect2">
<h3 id="_hbase_procedure_remote_dispatcher_threadpool_size"><a class="anchor" href="#_hbase_procedure_remote_dispatcher_threadpool_size"></a>203.1. hbase.procedure.remote.dispatcher.threadpool.size</h3>
<div class="paragraph">
<p>Defaults 128</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_procedure_remote_dispatcher_delay_msec"><a class="anchor" href="#_hbase_procedure_remote_dispatcher_delay_msec"></a>203.2. hbase.procedure.remote.dispatcher.delay.msec</h3>
<div class="paragraph">
<p>Default 150ms</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_procedure_remote_dispatcher_max_queue_size"><a class="anchor" href="#_hbase_procedure_remote_dispatcher_max_queue_size"></a>203.3. hbase.procedure.remote.dispatcher.max.queue.size</h3>
<div class="paragraph">
<p>Default 32</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_regionserver_rpc_startup_waittime"><a class="anchor" href="#_hbase_regionserver_rpc_startup_waittime"></a>203.4. hbase.regionserver.rpc.startup.waittime</h3>
<div class="paragraph">
<p>Default 60 seconds.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_tools"><a class="anchor" href="#_tools"></a>204. Tools</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBASE-15592 Print Procedure WAL Content</p>
</div>
<div class="paragraph">
<p>Patch in <a href="https://issues.apache.org/jira/browse/HBASE-18152">HBASE-18152</a> [AMv2] Corrupt Procedure WAL file; procedure data stored out of order <a href="https://issues.apache.org/jira/secure/attachment/12871066/reading_bad_wal.patch">https://issues.apache.org/jira/secure/attachment/12871066/reading_bad_wal.patch</a></p>
</div>
<div class="sect2">
<h3 id="_masterprocedureschedulerperformanceevaluation"><a class="anchor" href="#_masterprocedureschedulerperformanceevaluation"></a>204.1. MasterProcedureSchedulerPerformanceEvaluation</h3>
<div class="paragraph">
<p>Tool to test performance of locks and queues in procedure scheduler independently from other framework components. Run this after any substantial changes in proc system. Prints nice output:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>******************************************
Time - addBack : 5.0600sec
Ops/sec - addBack : 1.9M
Time - poll : 19.4590sec
Ops/sec - poll : 501.9K
Num Operations : 10000000
Completed : 10000006
Yield : 22025876
Num Tables : 5
Regions per table : 10
Operations type : both
Threads : 10
******************************************
Raw format for scripts
RESULT [num_ops=10000000, ops_type=both, num_table=5, regions_per_table=10, threads=10, num_yield=22025876, time_addback_ms=5060, time_poll_ms=19459]</pre>
</div>
</div>
</div>
</div>
</div>
<h1 id="zookeeper" class="sect0"><a class="anchor" href="#zookeeper"></a>ZooKeeper</h1>
<div class="openblock partintro">
<div class="content">
<div class="paragraph">
<p>A distributed Apache HBase installation depends on a running ZooKeeper cluster.
All participating nodes and clients need to be able to access the running ZooKeeper ensemble.
Apache HBase by default manages a ZooKeeper "cluster" for you.
It will start and stop the ZooKeeper ensemble as part of the HBase start/stop process.
You can also manage the ZooKeeper ensemble independent of HBase and just point HBase at the cluster it should use.
To toggle HBase management of ZooKeeper, use the <code>HBASE_MANAGES_ZK</code> variable in <em>conf/hbase-env.sh</em>.
This variable, which defaults to <code>true</code>, tells HBase whether to start/stop the ZooKeeper ensemble servers as part of HBase start/stop.</p>
</div>
<div class="paragraph">
<p>When HBase manages the ZooKeeper ensemble, you can specify ZooKeeper configuration directly in <em>conf/hbase-site.xml</em>.
A ZooKeeper configuration option can be set as a property in the HBase <em>hbase-site.xml</em> XML configuration file by prefacing the ZooKeeper option name with <code>hbase.zookeeper.property</code>.
For example, the <code>clientPort</code> setting in ZooKeeper can be changed by setting the <code>hbase.zookeeper.property.clientPort</code> property.
For all default values used by HBase, including ZooKeeper configuration, see <a href="#hbase_default_configurations">hbase default configurations</a>.
Look for the <code>hbase.zookeeper.property</code> prefix.
For the full list of ZooKeeper configurations, see ZooKeeper&#8217;s <em>zoo.cfg</em>.
HBase does not ship with a <em>zoo.cfg</em> so you will need to browse the <em>conf</em> directory in an appropriate ZooKeeper download.</p>
</div>
<div class="paragraph">
<p>You must at least list the ensemble servers in <em>hbase-site.xml</em> using the <code>hbase.zookeeper.quorum</code> property.
This property defaults to a single ensemble member at <code>localhost</code> which is not suitable for a fully distributed HBase.
(It binds to the local machine only and remote clients will not be able to connect).</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="title">How many ZooKeepers should I run?</div>
<div class="paragraph">
<p>You can run a ZooKeeper ensemble that comprises 1 node only but in production it is recommended that you run a ZooKeeper ensemble of 3, 5 or 7 machines; the more members an ensemble has, the more tolerant the ensemble is of host failures.
Also, run an odd number of machines.
In ZooKeeper, an even number of peers is supported, but it is normally not used because an even sized ensemble requires, proportionally, more peers to form a quorum than an odd sized ensemble requires.
For example, an ensemble with 4 peers requires 3 to form a quorum, while an ensemble with 5 also requires 3 to form a quorum.
Thus, an ensemble of 5 allows 2 peers to fail, and thus is more fault tolerant than the ensemble of 4, which allows only 1 down peer.</p>
</div>
<div class="paragraph">
<p>Give each ZooKeeper server around 1GB of RAM, and if possible, its own dedicated disk (A dedicated disk is the best thing you can do to ensure a performant ZooKeeper ensemble). For very heavily loaded clusters, run ZooKeeper servers on separate machines from RegionServers (DataNodes and TaskTrackers).</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>For example, to have HBase manage a ZooKeeper quorum on nodes <em>rs{1,2,3,4,5}.example.com</em>, bound to port 2222 (the default is 2181) ensure <code>HBASE_MANAGE_ZK</code> is commented out or set to <code>true</code> in <em>conf/hbase-env.sh</em> and then edit <em>conf/hbase-site.xml</em> and set <code>hbase.zookeeper.property.clientPort</code> and <code>hbase.zookeeper.quorum</code>.
You should also set <code>hbase.zookeeper.property.dataDir</code> to other than the default as the default has ZooKeeper persist data under <em>/tmp</em> which is often cleared on system restart.
In the example below we have ZooKeeper persist to <em>/user/local/zookeeper</em>.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"> &lt;configuration&gt;
...
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.property.clientPort&lt;/name&gt;
&lt;value&gt;<span class="integer">2222</span>&lt;/value&gt;
&lt;description&gt;Property from ZooKeeper<span class="string"><span class="delimiter">'</span><span class="content">s config zoo.cfg.
The port at which the clients will connect.
&lt;/description&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.quorum&lt;/name&gt;
&lt;value&gt;rs1.example.com,rs2.example.com,rs3.example.com,rs4.example.com,rs5.example.com&lt;/value&gt;
&lt;description&gt;Comma separated list of servers in the ZooKeeper Quorum.
For example, &quot;host1.mydomain.com,host2.mydomain.com,host3.mydomain.com&quot;.
By default this is set to localhost for local and pseudo-distributed modes
of operation. For a fully-distributed setup, this should be set to a full
list of ZooKeeper quorum servers. If HBASE_MANAGES_ZK is set in hbase-env.sh
this is the list of servers which we will start/stop ZooKeeper on.
&lt;/description&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.property.dataDir&lt;/name&gt;
&lt;value&gt;/usr/local/zookeeper&lt;/value&gt;
&lt;description&gt;Property from ZooKeeper</span><span class="delimiter">'</span></span>s config zoo.cfg.
The directory where the snapshot is stored.
&lt;/description&gt;
&lt;/property&gt;
...
&lt;/configuration&gt;</code></pre>
</div>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">What version of ZooKeeper should I use?</div>
<div class="paragraph">
<p>The newer version, the better. ZooKeeper 3.4.x is required as of HBase 1.0.0</p>
</div>
</td>
</tr>
</table>
</div>
<div class="admonitionblock caution">
<table>
<tr>
<td class="icon">
<i class="fa icon-caution" title="Caution"></i>
</td>
<td class="content">
<div class="title">ZooKeeper Maintenance</div>
<div class="paragraph">
<p>Be sure to set up the data dir cleaner described under <a href="https://zookeeper.apache.org/doc/r3.1.2/zookeeperAdmin.html#sc_maintenance">ZooKeeper
Maintenance</a> else you could have 'interesting' problems a couple of months in; i.e.
zookeeper could start dropping sessions if it has to run through a directory of hundreds of thousands of logs which is wont to do around leader reelection time&#8201;&#8212;&#8201;a process rare but run on occasion whether because a machine is dropped or happens to hiccup.</p>
</div>
</td>
</tr>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_using_existing_zookeeper_ensemble"><a class="anchor" href="#_using_existing_zookeeper_ensemble"></a>205. Using existing ZooKeeper ensemble</h2>
<div class="sectionbody">
<div class="paragraph">
<p>To point HBase at an existing ZooKeeper cluster, one that is not managed by HBase, set <code>HBASE_MANAGES_ZK</code> in <em>conf/hbase-env.sh</em> to false</p>
</div>
<div class="listingblock">
<div class="content">
<pre> ...
# Tell HBase whether it should manage its own instance of ZooKeeper or not.
export HBASE_MANAGES_ZK=false</pre>
</div>
</div>
<div class="paragraph">
<p>Next set ensemble locations and client port, if non-standard, in <em>hbase-site.xml</em>.</p>
</div>
<div class="paragraph">
<p>When HBase manages ZooKeeper, it will start/stop the ZooKeeper servers as a part of the regular start/stop scripts.
If you would like to run ZooKeeper yourself, independent of HBase start/stop, you would do the following</p>
</div>
<div class="listingblock">
<div class="content">
<pre>${HBASE_HOME}/bin/hbase-daemons.sh {start,stop} zookeeper</pre>
</div>
</div>
<div class="paragraph">
<p>Note that you can use HBase in this manner to spin up a ZooKeeper cluster, unrelated to HBase.
Just make sure to set <code>HBASE_MANAGES_ZK</code> to <code>false</code> if you want it to stay up across HBase restarts so that when HBase shuts down, it doesn&#8217;t take ZooKeeper down with it.</p>
</div>
<div class="paragraph">
<p>For more information about running a distinct ZooKeeper cluster, see the ZooKeeper <a href="https://zookeeper.apache.org/doc/current/zookeeperStarted.html">Getting
Started Guide</a>.
Additionally, see the <a href="https://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7">ZooKeeper Wiki</a> or the <a href="https://zookeeper.apache.org/doc/r3.4.10/zookeeperAdmin.html#sc_zkMulitServerSetup">ZooKeeper
documentation</a> for more information on ZooKeeper sizing.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="zk.sasl.auth"><a class="anchor" href="#zk.sasl.auth"></a>206. SASL Authentication with ZooKeeper</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Newer releases of Apache HBase (&gt;= 0.92) will support connecting to a ZooKeeper Quorum that supports SASL authentication (which is available in ZooKeeper versions 3.4.0 or later).</p>
</div>
<div class="paragraph">
<p>This describes how to set up HBase to mutually authenticate with a ZooKeeper Quorum.
ZooKeeper/HBase mutual authentication (<a href="https://issues.apache.org/jira/browse/HBASE-2418">HBASE-2418</a>) is required as part of a complete secure HBase configuration (<a href="https://issues.apache.org/jira/browse/HBASE-3025">HBASE-3025</a>). For simplicity of explication, this section ignores additional configuration required (Secure HDFS and Coprocessor configuration). It&#8217;s recommended to begin with an HBase-managed ZooKeeper configuration (as opposed to a standalone ZooKeeper quorum) for ease of learning.</p>
</div>
<div class="sect2">
<h3 id="_operating_system_prerequisites"><a class="anchor" href="#_operating_system_prerequisites"></a>206.1. Operating System Prerequisites</h3>
<div class="paragraph">
<p>You need to have a working Kerberos KDC setup.
For each <code>$HOST</code> that will run a ZooKeeper server, you should have a principle <code>zookeeper/$HOST</code>.
For each such host, add a service key (using the <code>kadmin</code> or <code>kadmin.local</code> tool&#8217;s <code>ktadd</code> command) for <code>zookeeper/$HOST</code> and copy this file to <code>$HOST</code>, and make it readable only to the user that will run zookeeper on <code>$HOST</code>.
Note the location of this file, which we will use below as <em>$PATH_TO_ZOOKEEPER_KEYTAB</em>.</p>
</div>
<div class="paragraph">
<p>Similarly, for each <code>$HOST</code> that will run an HBase server (master or regionserver), you should have a principle: <code>hbase/$HOST</code>.
For each host, add a keytab file called <em>hbase.keytab</em> containing a service key for <code>hbase/$HOST</code>, copy this file to <code>$HOST</code>, and make it readable only to the user that will run an HBase service on <code>$HOST</code>.
Note the location of this file, which we will use below as <em>$PATH_TO_HBASE_KEYTAB</em>.</p>
</div>
<div class="paragraph">
<p>Each user who will be an HBase client should also be given a Kerberos principal.
This principal should usually have a password assigned to it (as opposed to, as with the HBase servers, a keytab file) which only this user knows.
The client&#8217;s principal&#8217;s <code>maxrenewlife</code> should be set so that it can be renewed enough so that the user can complete their HBase client processes.
For example, if a user runs a long-running HBase client process that takes at most 3 days, we might create this user&#8217;s principal within <code>kadmin</code> with: <code>addprinc -maxrenewlife 3days</code>.
The ZooKeeper client and server libraries manage their own ticket refreshment by running threads that wake up periodically to do the refreshment.</p>
</div>
<div class="paragraph">
<p>On each host that will run an HBase client (e.g. <code>hbase shell</code>), add the following file to the HBase home directory&#8217;s <em>conf</em> directory:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Client {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=<span class="predefined-constant">false</span>
useTicketCache=<span class="predefined-constant">true</span>;
};</code></pre>
</div>
</div>
<div class="paragraph">
<p>We&#8217;ll refer to this JAAS configuration file as <em>$CLIENT_CONF</em> below.</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_managed_zookeeper_configuration"><a class="anchor" href="#_hbase_managed_zookeeper_configuration"></a>206.2. HBase-managed ZooKeeper Configuration</h3>
<div class="paragraph">
<p>On each node that will run a zookeeper, a master, or a regionserver, create a <a href="http://docs.oracle.com/javase/7/docs/technotes/guides/security/jgss/tutorials/LoginConfigFile.html">JAAS</a> configuration file in the conf directory of the node&#8217;s <em>HBASE_HOME</em> directory that looks like the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Server {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=<span class="predefined-constant">true</span>
keyTab=<span class="string"><span class="delimiter">&quot;</span><span class="content">$PATH_TO_ZOOKEEPER_KEYTAB</span><span class="delimiter">&quot;</span></span>
storeKey=<span class="predefined-constant">true</span>
useTicketCache=<span class="predefined-constant">false</span>
principal=<span class="string"><span class="delimiter">&quot;</span><span class="content">zookeeper/$HOST</span><span class="delimiter">&quot;</span></span>;
};
Client {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=<span class="predefined-constant">true</span>
useTicketCache=<span class="predefined-constant">false</span>
keyTab=<span class="string"><span class="delimiter">&quot;</span><span class="content">$PATH_TO_HBASE_KEYTAB</span><span class="delimiter">&quot;</span></span>
principal=<span class="string"><span class="delimiter">&quot;</span><span class="content">hbase/$HOST</span><span class="delimiter">&quot;</span></span>;
};</code></pre>
</div>
</div>
<div class="paragraph">
<p>where the <em>$PATH_TO_HBASE_KEYTAB</em> and <em>$PATH_TO_ZOOKEEPER_KEYTAB</em> files are what you created above, and <code>$HOST</code> is the hostname for that node.</p>
</div>
<div class="paragraph">
<p>The <code>Server</code> section will be used by the ZooKeeper quorum server, while the <code>Client</code> section will be used by the HBase master and regionservers.
The path to this file should be substituted for the text <em>$HBASE_SERVER_CONF</em> in the <em>hbase-env.sh</em> listing below.</p>
</div>
<div class="paragraph">
<p>The path to this file should be substituted for the text <em>$CLIENT_CONF</em> in the <em>hbase-env.sh</em> listing below.</p>
</div>
<div class="paragraph">
<p>Modify your <em>hbase-env.sh</em> to include the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">export HBASE_OPTS=&quot;-Djava.security.auth.login.config=$CLIENT_CONF&quot;
export HBASE_MANAGES_ZK=true
export HBASE_ZOOKEEPER_OPTS=&quot;-Djava.security.auth.login.config=$HBASE_SERVER_CONF&quot;
export HBASE_MASTER_OPTS=&quot;-Djava.security.auth.login.config=$HBASE_SERVER_CONF&quot;
export HBASE_REGIONSERVER_OPTS=&quot;-Djava.security.auth.login.config=$HBASE_SERVER_CONF&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>where <em>$HBASE_SERVER_CONF</em> and <em>$CLIENT_CONF</em> are the full paths to the JAAS configuration files created above.</p>
</div>
<div class="paragraph">
<p>Modify your <em>hbase-site.xml</em> on each node that will run zookeeper, master or regionserver to contain:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">&lt;configuration&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.quorum&lt;/name&gt;
&lt;value&gt;<span class="error">$</span>ZK_NODES&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.cluster.distributed&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">true</span>&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.property.authProvider<span class="float">.1</span>&lt;/name&gt;
&lt;value&gt;org.apache.zookeeper.server.auth.SASLAuthenticationProvider&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.property.kerberos.removeHostFromPrincipal&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">true</span>&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.zookeeper.property.kerberos.removeRealmFromPrincipal&lt;/name&gt;
&lt;value&gt;<span class="predefined-constant">true</span>&lt;/value&gt;
&lt;/property&gt;
&lt;/configuration&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>where <code>$ZK_NODES</code> is the comma-separated list of hostnames of the ZooKeeper Quorum hosts.</p>
</div>
<div class="paragraph">
<p>Start your hbase cluster by running one or more of the following set of commands on the appropriate hosts:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>bin/hbase zookeeper start
bin/hbase master start
bin/hbase regionserver start</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_external_zookeeper_configuration"><a class="anchor" href="#_external_zookeeper_configuration"></a>206.3. External ZooKeeper Configuration</h3>
<div class="paragraph">
<p>Add a JAAS configuration file that looks like:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Client {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=<span class="predefined-constant">true</span>
useTicketCache=<span class="predefined-constant">false</span>
keyTab=<span class="string"><span class="delimiter">&quot;</span><span class="content">$PATH_TO_HBASE_KEYTAB</span><span class="delimiter">&quot;</span></span>
principal=<span class="string"><span class="delimiter">&quot;</span><span class="content">hbase/$HOST</span><span class="delimiter">&quot;</span></span>;
};</code></pre>
</div>
</div>
<div class="paragraph">
<p>where the <em>$PATH_TO_HBASE_KEYTAB</em> is the keytab created above for HBase services to run on this host, and <code>$HOST</code> is the hostname for that node.
Put this in the HBase home&#8217;s configuration directory.
We&#8217;ll refer to this file&#8217;s full pathname as <em>$HBASE_SERVER_CONF</em> below.</p>
</div>
<div class="paragraph">
<p>Modify your hbase-env.sh to include the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">export HBASE_OPTS=&quot;-Djava.security.auth.login.config=$CLIENT_CONF&quot;
export HBASE_MANAGES_ZK=false
export HBASE_MASTER_OPTS=&quot;-Djava.security.auth.login.config=$HBASE_SERVER_CONF&quot;
export HBASE_REGIONSERVER_OPTS=&quot;-Djava.security.auth.login.config=$HBASE_SERVER_CONF&quot;</code></pre>
</div>
</div>
<div class="paragraph">
<p>Modify your <em>hbase-site.xml</em> on each node that will run a master or regionserver to contain:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;configuration&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.quorum<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>$ZK_NODES<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.cluster.distributed<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.authProvider.1<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.zookeeper.server.auth.SASLAuthenticationProvider<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.kerberos.removeHostFromPrincipal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.zookeeper.property.kerberos.removeRealmFromPrincipal<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>true<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;/configuration&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>where <code>$ZK_NODES</code> is the comma-separated list of hostnames of the ZooKeeper Quorum hosts.</p>
</div>
<div class="paragraph">
<p>Also on each of these hosts, create a JAAS configuration file containing:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">Server {
com.sun.security.auth.module.Krb5LoginModule required
useKeyTab=<span class="predefined-constant">true</span>
keyTab=<span class="string"><span class="delimiter">&quot;</span><span class="content">$PATH_TO_ZOOKEEPER_KEYTAB</span><span class="delimiter">&quot;</span></span>
storeKey=<span class="predefined-constant">true</span>
useTicketCache=<span class="predefined-constant">false</span>
principal=<span class="string"><span class="delimiter">&quot;</span><span class="content">zookeeper/$HOST</span><span class="delimiter">&quot;</span></span>;
};</code></pre>
</div>
</div>
<div class="paragraph">
<p>where <code>$HOST</code> is the hostname of each Quorum host.
We will refer to the full pathname of this file as <em>$ZK_SERVER_CONF</em> below.</p>
</div>
<div class="paragraph">
<p>Start your ZooKeepers on each ZooKeeper Quorum host with:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">SERVER_JVMFLAGS=&quot;-Djava.security.auth.login.config=$ZK_SERVER_CONF&quot; bin/zkServer start</code></pre>
</div>
</div>
<div class="paragraph">
<p>Start your HBase cluster by running one or more of the following set of commands on the appropriate nodes:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>bin/hbase master start
bin/hbase regionserver start</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_zookeeper_server_authentication_log_output"><a class="anchor" href="#_zookeeper_server_authentication_log_output"></a>206.4. ZooKeeper Server Authentication Log Output</h3>
<div class="paragraph">
<p>If the configuration above is successful, you should see something similar to the following in your ZooKeeper server logs:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>11/12/05 22:43:39 INFO zookeeper.Login: successfully logged in.
11/12/05 22:43:39 INFO server.NIOServerCnxnFactory: binding to port 0.0.0.0/0.0.0.0:2181
11/12/05 22:43:39 INFO zookeeper.Login: TGT refresh thread started.
11/12/05 22:43:39 INFO zookeeper.Login: TGT valid starting at: Mon Dec 05 22:43:39 UTC 2011
11/12/05 22:43:39 INFO zookeeper.Login: TGT expires: Tue Dec 06 22:43:39 UTC 2011
11/12/05 22:43:39 INFO zookeeper.Login: TGT refresh sleeping until: Tue Dec 06 18:36:42 UTC 2011
..
11/12/05 22:43:59 INFO auth.SaslServerCallbackHandler:
Successfully authenticated client: authenticationID=hbase/ip-10-166-175-249.us-west-1.compute.internal@HADOOP.LOCALDOMAIN;
authorizationID=hbase/ip-10-166-175-249.us-west-1.compute.internal@HADOOP.LOCALDOMAIN.
11/12/05 22:43:59 INFO auth.SaslServerCallbackHandler: Setting authorizedID: hbase
11/12/05 22:43:59 INFO server.ZooKeeperServer: adding SASL authorization for authorizationID: hbase</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_zookeeper_client_authentication_log_output"><a class="anchor" href="#_zookeeper_client_authentication_log_output"></a>206.5. ZooKeeper Client Authentication Log Output</h3>
<div class="paragraph">
<p>On the ZooKeeper client side (HBase master or regionserver), you should see something similar to the following:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>11/12/05 22:43:59 INFO zookeeper.ZooKeeper: Initiating client connection, connectString=ip-10-166-175-249.us-west-1.compute.internal:2181 sessionTimeout=180000 watcher=master:60000
11/12/05 22:43:59 INFO zookeeper.ClientCnxn: Opening socket connection to server /10.166.175.249:2181
11/12/05 22:43:59 INFO zookeeper.RecoverableZooKeeper: The identifier of this process is 14851@ip-10-166-175-249
11/12/05 22:43:59 INFO zookeeper.Login: successfully logged in.
11/12/05 22:43:59 INFO client.ZooKeeperSaslClient: Client will use GSSAPI as SASL mechanism.
11/12/05 22:43:59 INFO zookeeper.Login: TGT refresh thread started.
11/12/05 22:43:59 INFO zookeeper.ClientCnxn: Socket connection established to ip-10-166-175-249.us-west-1.compute.internal/10.166.175.249:2181, initiating session
11/12/05 22:43:59 INFO zookeeper.Login: TGT valid starting at: Mon Dec 05 22:43:59 UTC 2011
11/12/05 22:43:59 INFO zookeeper.Login: TGT expires: Tue Dec 06 22:43:59 UTC 2011
11/12/05 22:43:59 INFO zookeeper.Login: TGT refresh sleeping until: Tue Dec 06 18:30:37 UTC 2011
11/12/05 22:43:59 INFO zookeeper.ClientCnxn: Session establishment complete on server ip-10-166-175-249.us-west-1.compute.internal/10.166.175.249:2181, sessionid = 0x134106594320000, negotiated timeout = 180000</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_configuration_from_scratch"><a class="anchor" href="#_configuration_from_scratch"></a>206.6. Configuration from Scratch</h3>
<div class="paragraph">
<p>This has been tested on the current standard Amazon Linux AMI.
First setup KDC and principals as described above.
Next checkout code and run a sanity check.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>git clone https://gitbox.apache.org/repos/asf/hbase.git
cd hbase
mvn clean test -Dtest=TestZooKeeperACL</pre>
</div>
</div>
<div class="paragraph">
<p>Then configure HBase as described above.
Manually edit target/cached_classpath.txt (see below):</p>
</div>
<div class="listingblock">
<div class="content">
<pre>bin/hbase zookeeper &amp;
bin/hbase master &amp;
bin/hbase regionserver &amp;</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_future_improvements"><a class="anchor" href="#_future_improvements"></a>206.7. Future improvements</h3>
<div class="sect3">
<h4 id="_fix_target_cached_classpath_txt"><a class="anchor" href="#_fix_target_cached_classpath_txt"></a>206.7.1. Fix target/cached_classpath.txt</h4>
<div class="paragraph">
<p>You must override the standard hadoop-core jar file from the <code>target/cached_classpath.txt</code> file with the version containing the HADOOP-7070 fix.
You can use the following script to do this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>echo `find ~/.m2 -name "*hadoop-core*7070*SNAPSHOT.jar"` ':' `cat target/cached_classpath.txt` | sed 's/ //g' &gt; target/tmp.txt
mv target/tmp.txt target/cached_classpath.txt</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_set_jaas_configuration_programmatically"><a class="anchor" href="#_set_jaas_configuration_programmatically"></a>206.7.2. Set JAAS configuration programmatically</h4>
<div class="paragraph">
<p>This would avoid the need for a separate Hadoop jar that fixes <a href="https://issues.apache.org/jira/browse/HADOOP-7070">HADOOP-7070</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="_elimination_of_code_kerberos_removehostfromprincipal_code_and_kerberos_removerealmfromprincipal"><a class="anchor" href="#_elimination_of_code_kerberos_removehostfromprincipal_code_and_kerberos_removerealmfromprincipal"></a>206.7.3. Elimination of <code>kerberos.removeHostFromPrincipal</code> and`kerberos.removeRealmFromPrincipal`</h4>
</div>
</div>
</div>
</div>
<h1 id="community" class="sect0"><a class="anchor" href="#community"></a>Community</h1>
<div class="sect1">
<h2 id="_decisions"><a class="anchor" href="#_decisions"></a>207. Decisions</h2>
<div class="sectionbody">
<div class="paragraph">
<div class="title">Feature Branches</div>
<p>Feature Branches are easy to make.
You do not have to be a committer to make one.
Just request the name of your branch be added to JIRA up on the developer&#8217;s mailing list and a committer will add it for you.
Thereafter you can file issues against your feature branch in Apache HBase JIRA.
Your code you keep elsewhere&#8201;&#8212;&#8201;it should be public so it can be observed&#8201;&#8212;&#8201;and you can update dev mailing list on progress.
When the feature is ready for commit, 3 +1s from committers will get your feature merged.
See <a href="http://search-hadoop.com/m/asM982C5FkS1">HBase, mail # dev - Thoughts
about large feature dev branches</a></p>
</div>
<div id="hbase.fix.version.in.jira" class="paragraph">
<div class="title">How to set fix version in JIRA on issue resolve</div>
<p>Here is how <a href="http://search-hadoop.com/m/azemIi5RCJ1">we agreed</a> to set versions in JIRA when we resolve an issue.
If master is going to be 2.0.0, and branch-1 1.4.0 then:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Commit only to master: Mark with 2.0.0</p>
</li>
<li>
<p>Commit to branch-1 and master: Mark with 2.0.0, and 1.4.0</p>
</li>
<li>
<p>Commit to branch-1.3, branch-1, and master: Mark with 2.0.0, 1.4.0, and 1.3.x</p>
</li>
<li>
<p>Commit site fixes: no version</p>
</li>
</ul>
</div>
<div id="hbase.when.to.close.jira" class="paragraph">
<div class="title">Policy on when to set a RESOLVED JIRA as CLOSED</div>
<p>We <a href="http://search-hadoop.com/m/4cIKs1iwXMS1">agreed</a> that for issues that list multiple releases in their <em>Fix Version/s</em> field, CLOSE the issue on the release of any of the versions listed; subsequent change to the issue must happen in a new JIRA.</p>
</div>
<div id="no.permanent.state.in.zk" class="paragraph">
<div class="title">Only transient state in ZooKeeper!</div>
<p>You should be able to kill the data in zookeeper and hbase should ride over it recreating the zk content as it goes.
This is an old adage around these parts.
We just made note of it now.
We also are currently in violation of this basic tenet&#8201;&#8212;&#8201;replication at least keeps permanent state in zk&#8201;&#8212;&#8201;but we are working to undo this breaking of a golden rule.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="community.roles"><a class="anchor" href="#community.roles"></a>208. Community Roles</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_release_managers"><a class="anchor" href="#_release_managers"></a>208.1. Release Managers</h3>
<div class="paragraph">
<p>Each maintained release branch has a release manager, who volunteers to coordinate new features and bug fixes are backported to that release.
The release managers are <a href="https://hbase.apache.org/team-list.html">committers</a>.
If you would like your feature or bug fix to be included in a given release, communicate with that release manager.
If this list goes out of date or you can&#8217;t reach the listed person, reach out to someone else on the list.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
End-of-life releases are not included in this list.
</td>
</tr>
</table>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 22. Release Managers</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Release</th>
<th class="tableblock halign-left valign-top">Release Manager</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">1.3</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Mikhail Antonov</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">1.4</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Andrew Purtell</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">2.0</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Michael Stack</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">2.1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Duo Zhang</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">2.2</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Guanghao Zhang</p></td>
</tr>
</tbody>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.commit.msg.format"><a class="anchor" href="#hbase.commit.msg.format"></a>209. Commit Message format</h2>
<div class="sectionbody">
<div class="paragraph">
<p>We <a href="http://search-hadoop.com/m/Gwxwl10cFHa1">agreed</a> to the following Git commit message format:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">HBASE-xxxxx &lt;title&gt;. (&lt;contributor&gt;)</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the person making the commit is the contributor, leave off the '(&lt;contributor&gt;)' element.</p>
</div>
</div>
</div>
<h1 id="_appendix" class="sect0"><a class="anchor" href="#_appendix"></a>Appendix</h1>
<div class="sect1">
<h2 id="appendix_contributing_to_documentation"><a class="anchor" href="#appendix_contributing_to_documentation"></a>Appendix A: Contributing to Documentation</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The Apache HBase project welcomes contributions to all aspects of the project,
including the documentation.</p>
</div>
<div class="paragraph">
<p>In HBase, documentation includes the following areas, and probably some others:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The <a href="https://hbase.apache.org/book.html">HBase Reference
Guide</a> (this book)</p>
</li>
<li>
<p>The <a href="https://hbase.apache.org/">HBase website</a></p>
</li>
<li>
<p>API documentation</p>
</li>
<li>
<p>Command-line utility output and help text</p>
</li>
<li>
<p>Web UI strings, explicit help text, context-sensitive strings, and others</p>
</li>
<li>
<p>Log messages</p>
</li>
<li>
<p>Comments in source files, configuration files, and others</p>
</li>
<li>
<p>Localization of any of the above into target languages other than English</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>No matter which area you want to help out with, the first step is almost always
to download (typically by cloning the Git repository) and familiarize yourself
with the HBase source code. For information on downloading and building the source,
see <a href="#developer">developer</a>.</p>
</div>
<div class="sect2">
<h3 id="_contributing_to_documentation_or_other_strings"><a class="anchor" href="#_contributing_to_documentation_or_other_strings"></a>A.1. Contributing to Documentation or Other Strings</h3>
<div class="paragraph">
<p>If you spot an error in a string in a UI, utility, script, log message, or elsewhere,
or you think something could be made more clear, or you think text needs to be added
where it doesn&#8217;t currently exist, the first step is to file a JIRA. Be sure to set
the component to <code>Documentation</code> in addition to any other involved components. Most
components have one or more default owners, who monitor new issues which come into
those queues. Regardless of whether you feel able to fix the bug, you should still
file bugs where you see them.</p>
</div>
<div class="paragraph">
<p>If you want to try your hand at fixing your newly-filed bug, assign it to yourself.
You will need to clone the HBase Git repository to your local system and work on
the issue there. When you have developed a potential fix, submit it for review.
If it addresses the issue and is seen as an improvement, one of the HBase committers
will commit it to one or more branches, as appropriate.</p>
</div>
<div id="submit_doc_patch_procedure" class="paragraph">
<div class="title">Procedure: Suggested Work flow for Submitting Patches</div>
<p>This procedure goes into more detail than Git pros will need, but is included
in this appendix so that people unfamiliar with Git can feel confident contributing
to HBase while they learn.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>If you have not already done so, clone the Git repository locally.
You only need to do this once.</p>
</li>
<li>
<p>Fairly often, pull remote changes into your local repository by using the
<code>git pull</code> command, while your tracking branch is checked out.</p>
</li>
<li>
<p>For each issue you work on, create a new branch.
One convention that works well for naming the branches is to name a given branch
the same as the JIRA it relates to:</p>
<div class="listingblock">
<div class="content">
<pre>$ git checkout -b HBASE-123456</pre>
</div>
</div>
</li>
<li>
<p>Make your suggested changes on your branch, committing your changes to your
local repository often. If you need to switch to working on a different issue,
remember to check out the appropriate branch.</p>
</li>
<li>
<p>When you are ready to submit your patch, first be sure that HBase builds cleanly
and behaves as expected in your modified branch.</p>
</li>
<li>
<p>If you have made documentation changes, be sure the documentation and website
builds by running <code>mvn clean site</code>.</p>
</li>
<li>
<p>If it takes you several days or weeks to implement your fix, or you know that
the area of the code you are working in has had a lot of changes lately, make
sure you rebase your branch against the remote master and take care of any conflicts
before submitting your patch.</p>
<div class="listingblock">
<div class="content">
<pre>$ git checkout HBASE-123456
$ git rebase origin/master</pre>
</div>
</div>
</li>
<li>
<p>Generate your patch against the remote master. Run the following command from
the top level of your git repository (usually called <code>hbase</code>):</p>
<div class="listingblock">
<div class="content">
<pre>$ git format-patch --stdout origin/master &gt; HBASE-123456.patch</pre>
</div>
</div>
<div class="paragraph">
<p>The name of the patch should contain the JIRA ID.</p>
</div>
</li>
<li>
<p>Look over the patch file to be sure that you did not change any additional files
by accident and that there are no other surprises.</p>
</li>
<li>
<p>When you are satisfied, attach the patch to the JIRA and click the
<b class="button">Patch Available</b> button. A reviewer will review your patch.</p>
</li>
<li>
<p>If you need to submit a new version of the patch, leave the old one on the
JIRA and add a version number to the name of the new patch.</p>
</li>
<li>
<p>After a change has been committed, there is no need to keep your local branch around.</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_editing_the_hbase_website"><a class="anchor" href="#_editing_the_hbase_website"></a>A.2. Editing the HBase Website</h3>
<div class="paragraph">
<p>The source for the HBase website is in the HBase source, in the <em>src/site/</em> directory.
Within this directory, source for the individual pages is in the <em>xdocs/</em> directory,
and images referenced in those pages are in the <em>resources/images/</em> directory.
This directory also stores images used in the HBase Reference Guide.</p>
</div>
<div class="paragraph">
<p>The website&#8217;s pages are written in an HTML-like XML dialect called xdoc, which
has a reference guide at
<a href="https://maven.apache.org/archives/maven-1.x/plugins/xdoc/reference/xdocs.html" class="bare">https://maven.apache.org/archives/maven-1.x/plugins/xdoc/reference/xdocs.html</a>.
You can edit these files in a plain-text editor, an IDE, or an XML editor such
as XML Mind XML Editor (XXE) or Oxygen XML Author.</p>
</div>
<div class="paragraph">
<p>To preview your changes, build the website using the <code>mvn clean site -DskipTests</code>
command. The HTML output resides in the <em>target/site/</em> directory.
When you are satisfied with your changes, follow the procedure in
<a href="#submit_doc_patch_procedure">submit doc patch procedure</a> to submit your patch.</p>
</div>
</div>
<div class="sect2">
<h3 id="website_publish"><a class="anchor" href="#website_publish"></a>A.3. Publishing the HBase Website and Documentation</h3>
<div class="paragraph">
<p>HBase uses the ASF&#8217;s <code>gitpubsub</code> mechanism. A Jenkins job runs the
<code>dev-support/jenkins-scripts/generate-hbase-website.sh</code> script, which runs the
<code>mvn clean site site:stage</code> against the <code>master</code> branch of the <code>hbase</code>
repository and commits the built artifacts to the <code>asf-site</code> branch of the
<code>hbase-site</code> repository. When the commit is pushed, the website is redeployed
automatically. If the script encounters an error, an email is sent to the
developer mailing list. You can run the script manually or examine it to see the
steps involved.</p>
</div>
</div>
<div class="sect2">
<h3 id="website_check_links"><a class="anchor" href="#website_check_links"></a>A.4. Checking the HBase Website for Broken Links</h3>
<div class="paragraph">
<p>A Jenkins job runs periodically to check HBase website for broken links, using
the <code>dev-support/jenkins-scripts/check-website-links.sh</code> script. This script
uses a tool called <code>linklint</code> to check for bad links and create a report. If
broken links are found, an email is sent to the developer mailing list. You can
run the script manually or examine it to see the steps involved.</p>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_reference_guide_style_guide_and_cheat_sheet"><a class="anchor" href="#_hbase_reference_guide_style_guide_and_cheat_sheet"></a>A.5. HBase Reference Guide Style Guide and Cheat Sheet</h3>
<div class="paragraph">
<p>The HBase Reference Guide is written in Asciidoc and built using <a href="http://asciidoctor.org">AsciiDoctor</a>.
The following cheat sheet is included for your reference. More nuanced and comprehensive documentation
is available at <a href="http://asciidoctor.org/docs/user-manual/" class="bare">http://asciidoctor.org/docs/user-manual/</a>.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 23. AsciiDoc Cheat Sheet</caption>
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Element Type</th>
<th class="tableblock halign-left valign-top">Desired Rendering</th>
<th class="tableblock halign-left valign-top">How to do it</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A paragraph</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">a paragraph</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Just type some text with a blank line at the top and bottom.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Add line breaks within a paragraph without adding blank lines</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Manual line breaks</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>This will break + at the plus sign. Or prefix the whole paragraph with a line containing '[%hardbreaks]'</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Give a title to anything</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Colored italic bold differently-sized text</p></td>
<td class="tableblock halign-left valign-top"><div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">In-Line Code or commands</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">monospace</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>`text`</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">In-line literal content (things to be typed exactly as shown)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">bold mono</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>*`typethis`*</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">In-line replaceable content (things to substitute with your own values)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">bold italic mono</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>*_typesomething_*</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Code blocks with highlighting</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">monospace, highlighted, preserve space</p></td>
<td class="tableblock halign-left valign-top"><div><div class="literalblock">
<div class="content">
<pre>[source,java]
----
myAwesomeCode() {
}
----</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Code block included from a separate file</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">included just as though it were part of the main file</p></td>
<td class="tableblock halign-left valign-top"><div><div class="literalblock">
<div class="content">
<pre>[source,ruby]
----
include\::path/to/app.rb[]
----</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Include only part of a separate file</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Similar to Javadoc</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>See <a href="http://asciidoctor.org/docs/user-manual/#by-tagged-regions" class="bare">http://asciidoctor.org/docs/user-manual/#by-tagged-regions</a></p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Filenames, directory names, new terms</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">italic</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>_hbase-default.xml_</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">External naked URLs</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A link with the URL as link text</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>link:http://www.google.com</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">External URLs with text</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A link with arbitrary link text</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>link:http://www.google.com[Google]</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Create an internal anchor to cross-reference</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">not rendered</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>[[anchor_name]]</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Cross-reference an existing anchor using its default title</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">an internal hyperlink using the element title if available, otherwise using the anchor name</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>&lt;&lt;anchor_name&gt;&gt;</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Cross-reference an existing anchor using custom text</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">an internal hyperlink using arbitrary text</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>&lt;&lt;anchor_name,Anchor Text&gt;&gt;</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A block image</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The image with alt text</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>image::sunset.jpg[Alt Text]</pre>
</div>
</div>
<div class="paragraph">
<p>(put the image in the src/site/resources/images directory)</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">An inline image</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The image with alt text, as part of the text flow</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>image:sunset.jpg [Alt Text]</pre>
</div>
</div>
<div class="paragraph">
<p>(only one colon)</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Link to a remote image</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">show an image hosted elsewhere</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>image::http://inkscape.org/doc/examples/tux.svg[Tux,250,350]</pre>
</div>
</div>
<div class="paragraph">
<p>(or <code>image:</code>)</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Add dimensions or a URL to the image</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">depends</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>inside the brackets after the alt text, specify width, height and/or link="http://my_link.com"</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A footnote</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">subscript link which takes you to the footnote</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>Some text.footnote:[The footnote text.]</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A note or warning with no title</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The admonition image followed by the admonition</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>NOTE:My note here</pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre>WARNING:My warning here</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A complex note</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The note has a title and/or multiple paragraphs and/or code blocks or lists, etc</p></td>
<td class="tableblock halign-left valign-top"><div><div class="literalblock">
<div class="content">
<pre>.The Title
[NOTE]
====
Here is the note text. Everything until the second set of four equals signs is part of the note.
----
some source code
----
====</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Bullet lists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">bullet lists</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>* list item 1</pre>
</div>
</div>
<div class="paragraph">
<p>(see <a href="http://asciidoctor.org/docs/user-manual/#unordered-lists" class="bare">http://asciidoctor.org/docs/user-manual/#unordered-lists</a>)</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Numbered lists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">numbered list</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>. list item 2</pre>
</div>
</div>
<div class="paragraph">
<p>(see <a href="http://asciidoctor.org/docs/user-manual/#ordered-lists" class="bare">http://asciidoctor.org/docs/user-manual/#ordered-lists</a>)</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Checklists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Checked or unchecked boxes</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Checked:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>- [*]</pre>
</div>
</div>
<div class="paragraph">
<p>Unchecked:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>- [ ]</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Multiple levels of lists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">bulleted or numbered or combo</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>. Numbered (1), at top level
* Bullet (2), nested under 1
* Bullet (3), nested under 1
. Numbered (4), at top level
* Bullet (5), nested under 4
** Bullet (6), nested under 5
- [x] Checked (7), at top level</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Labelled lists / variablelists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">a list item title or summary followed by content</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>Title:: content
Title::
content</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Sidebars, quotes, or other blocks of text</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">a block of text, formatted differently from the default</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>Delimited using different delimiters,
see <a href="http://asciidoctor.org/docs/user-manual/#built-in-blocks-summary" class="bare">http://asciidoctor.org/docs/user-manual/#built-in-blocks-summary</a>.
Some of the examples above use delimiters like ...., ----,====.</p>
</div>
<div class="literalblock">
<div class="content">
<pre>[example]
====
This is an example block.
====
[source]
----
This is a source block.
----
[note]
====
This is a note block.
====
[quote]
____
This is a quote block.
____</pre>
</div>
</div>
<div class="paragraph">
<p>If you want to insert literal Asciidoc content that keeps being interpreted, when in doubt, use eight dots as the delimiter at the top and bottom.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Nested Sections</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">chapter, section, sub-section, etc</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>= Book (or chapter if the chapter can be built alone, see the leveloffset info below)
== Chapter (or section if the chapter is standalone)
=== Section (or subsection, etc)
==== Subsection</pre>
</div>
</div>
<div class="paragraph">
<p>and so on up to 6 levels (think carefully about going deeper than 4 levels, maybe you can just titled paragraphs or lists instead). Note that you can include a book inside another book by adding the <code>:leveloffset:+1</code> macro directive directly before your include, and resetting it to 0 directly after. See the <em>book.adoc</em> source for examples, as this is how this guide handles chapters. <strong>Don&#8217;t do it for prefaces, glossaries, appendixes, or other special types of chapters.</strong></p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Include one file from another</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Content is included as though it were inline</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>include::[/path/to/file.adoc]</pre>
</div>
</div>
<div class="paragraph">
<p>For plenty of examples. see <em>book.adoc</em>.</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">A table</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">a table</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p>See <a href="http://asciidoctor.org/docs/user-manual/#tables" class="bare">http://asciidoctor.org/docs/user-manual/#tables</a>. Generally rows are separated by newlines and columns by pipes</p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Comment out a single line</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A line is skipped during rendering</p></td>
<td class="tableblock halign-left valign-top"><div><div class="paragraph">
<p><code>// This line won&#8217;t show up</code></p>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Comment out a block</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A section of the file is skipped during rendering</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>////
Nothing between the slashes will show up.
////</pre>
</div>
</div></div></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Highlight text for review</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">text shows up with yellow background</p></td>
<td class="tableblock halign-left valign-top"><div><div class="listingblock">
<div class="content">
<pre>Test between #hash marks# is highlighted yellow.</pre>
</div>
</div></div></td>
</tr>
</tbody>
</table>
</div>
<div class="sect2">
<h3 id="_auto_generated_content"><a class="anchor" href="#_auto_generated_content"></a>A.6. Auto-Generated Content</h3>
<div class="paragraph">
<p>Some parts of the HBase Reference Guide, most notably <a href="#config.files">config.files</a>,
are generated automatically, so that this area of the documentation stays in
sync with the code. This is done by means of an XSLT transform, which you can examine
in the source at <em>src/main/xslt/configuration_to_asciidoc_chapter.xsl</em>. This
transforms the <em>hbase-common/src/main/resources/hbase-default.xml</em> file into an
Asciidoc output which can be included in the Reference Guide.</p>
</div>
<div class="paragraph">
<p>Sometimes, it is necessary to add configuration parameters or modify their descriptions.
Make the modifications to the source file, and they will be included in the
Reference Guide when it is rebuilt.</p>
</div>
<div class="paragraph">
<p>It is possible that other types of content can and will be automatically generated
from HBase source files in the future.</p>
</div>
</div>
<div class="sect2">
<h3 id="_images_in_the_hbase_reference_guide"><a class="anchor" href="#_images_in_the_hbase_reference_guide"></a>A.7. Images in the HBase Reference Guide</h3>
<div class="paragraph">
<p>You can include images in the HBase Reference Guide. It is important to include
an image title if possible, and alternate text always. This allows screen readers
to navigate to the image and also provides alternative text for the image.
The following is an example of an image with a title and alternate text. Notice
the double colon.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="asciidoc">.My Image Title
image::sunset.jpg[Alt Text]</code></pre>
</div>
</div>
<div class="paragraph">
<p>Here is an example of an inline image with alternate text. Notice the single colon.
Inline images cannot have titles. They are generally small images like GUI buttons.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="asciidoc">image:sunset.jpg[Alt Text]</code></pre>
</div>
</div>
<div class="paragraph">
<p>When doing a local build, save the image to the <em>src/site/resources/images/</em> directory.
When you link to the image, do not include the directory portion of the path.
The image will be copied to the appropriate target location during the build of the output.</p>
</div>
<div class="paragraph">
<p>When you submit a patch which includes adding an image to the HBase Reference Guide,
attach the image to the JIRA. If the committer asks where the image should be
committed, it should go into the above directory.</p>
</div>
</div>
<div class="sect2">
<h3 id="_adding_a_new_chapter_to_the_hbase_reference_guide"><a class="anchor" href="#_adding_a_new_chapter_to_the_hbase_reference_guide"></a>A.8. Adding a New Chapter to the HBase Reference Guide</h3>
<div class="paragraph">
<p>If you want to add a new chapter to the HBase Reference Guide, the easiest way
is to copy an existing chapter file, rename it, and change the ID (in double
brackets) and title. Chapters are located in the <em>src/main/asciidoc/_chapters/</em>
directory.</p>
</div>
<div class="paragraph">
<p>Delete the existing content and create the new content. Then open the
<em>src/main/asciidoc/book.adoc</em> file, which is the main file for the HBase Reference
Guide, and copy an existing <code>include</code> element to include your new chapter in the
appropriate location. Be sure to add your new file to your Git repository before
creating your patch.</p>
</div>
<div class="paragraph">
<p>When in doubt, check to see how other files have been included.</p>
</div>
</div>
<div class="sect2">
<h3 id="_common_documentation_issues"><a class="anchor" href="#_common_documentation_issues"></a>A.9. Common Documentation Issues</h3>
<div class="paragraph">
<p>The following documentation issues come up often. Some of these are preferences,
but others can create mysterious build errors or other problems.</p>
</div>
<div class="qlist qanda">
<ol>
<li>
<p><em>Isolate Changes for Easy Diff Review.</em></p>
<p>Be careful with pretty-printing or re-formatting an entire XML file, even if
the formatting has degraded over time. If you need to reformat a file, do that
in a separate JIRA where you do not change any content. Be careful because some
XML editors do a bulk-reformat when you open a new file, especially if you use
GUI mode in the editor.</p>
</li>
<li>
<p><em>Syntax Highlighting</em></p>
<p>The HBase Reference Guide uses <code>coderay</code> for syntax highlighting. To enable
syntax highlighting for a given code listing, use the following type of syntax:</p>
<div class="literalblock">
<div class="content">
<pre>[source,xml]
----
&lt;name&gt;My Name&lt;/name&gt;
----</pre>
</div>
</div>
<div class="paragraph">
<p>Several syntax types are supported. The most interesting ones for the HBase
Reference Guide are <code>java</code>, <code>xml</code>, <code>sql</code>, and <code>bash</code>.</p>
</div>
</li>
</ol>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="faq"><a class="anchor" href="#faq"></a>Appendix B: FAQ</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="_general"><a class="anchor" href="#_general"></a>B.1. General</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">When should I use HBase?</dt>
<dd>
<p>See <a href="#arch.overview">Overview</a> in the Architecture chapter.</p>
</dd>
<dt class="hdlist1">Are there other HBase FAQs?</dt>
<dd>
<p>See the FAQ that is up on the wiki, <a href="https://wiki.apache.org/hadoop/Hbase/FAQ">HBase Wiki FAQ</a>.</p>
</dd>
<dt class="hdlist1">Does HBase support SQL?</dt>
<dd>
<p>Not really. SQL-ish support for HBase via <a href="https://hive.apache.org/">Hive</a> is in development, however Hive is based on MapReduce which is not generally suitable for low-latency requests. See the <a href="#datamodel">Data Model</a> section for examples on the HBase client.</p>
</dd>
<dt class="hdlist1">How can I find examples of NoSQL/HBase?</dt>
<dd>
<p>See the link to the BigTable paper in <a href="#other.info">Other Information About HBase</a>, as well as the other papers.</p>
</dd>
<dt class="hdlist1">What is the history of HBase?</dt>
<dd>
<p>See <a href="#hbase.history">hbase.history</a>.</p>
</dd>
<dt class="hdlist1">Why are the cells above 10MB not recommended for HBase?</dt>
<dd>
<p>Large cells don&#8217;t fit well into HBase&#8217;s approach to buffering data. First, the large cells bypass the MemStoreLAB when they are written. Then, they cannot be cached in the L2 block cache during read operations. Instead, HBase has to allocate on-heap memory for them each time. This can have a significant impact on the garbage collector within the RegionServer process.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_upgrading"><a class="anchor" href="#_upgrading"></a>B.2. Upgrading</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How do I upgrade Maven-managed projects from HBase 0.94 to HBase 0.96+?</dt>
<dd>
<p>In HBase 0.96, the project moved to a modular structure. Adjust your project&#8217;s dependencies to rely upon the <code>hbase-client</code> module or another module as appropriate, rather than a single JAR. You can model your Maven dependency after one of the following, depending on your targeted version of HBase. See Section 3.5, “Upgrading from 0.94.x to 0.96.x” or Section 3.3, “Upgrading from 0.96.x to 0.98.x” for more information.</p>
<div class="listingblock">
<div class="title">Maven Dependency for HBase 0.98</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.hbase<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>hbase-client<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>0.98.5-hadoop2<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="title">Maven Dependency for HBase 0.96</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.hbase<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>hbase-client<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>0.96.2-hadoop2<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
<div class="listingblock">
<div class="title">Maven Dependency for HBase 0.94</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;dependency&gt;</span>
<span class="tag">&lt;groupId&gt;</span>org.apache.hbase<span class="tag">&lt;/groupId&gt;</span>
<span class="tag">&lt;artifactId&gt;</span>hbase<span class="tag">&lt;/artifactId&gt;</span>
<span class="tag">&lt;version&gt;</span>0.94.3<span class="tag">&lt;/version&gt;</span>
<span class="tag">&lt;/dependency&gt;</span></code></pre>
</div>
</div>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_architecture_2"><a class="anchor" href="#_architecture_2"></a>B.3. Architecture</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How does HBase handle Region-RegionServer assignment and locality?</dt>
<dd>
<p>See <a href="#regions.arch">Regions</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_configuration_2"><a class="anchor" href="#_configuration_2"></a>B.4. Configuration</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How can I get started with my first cluster?</dt>
<dd>
<p>See <a href="#quickstart">Quick Start - Standalone HBase</a>.</p>
</dd>
<dt class="hdlist1">Where can I learn about the rest of the configuration options?</dt>
<dd>
<p>See <a href="#configuration">Apache HBase Configuration</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_schema_design_data_access"><a class="anchor" href="#_schema_design_data_access"></a>B.5. Schema Design / Data Access</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How should I design my schema in HBase?</dt>
<dd>
<p>See <a href="#datamodel">Data Model</a> and <a href="#schema">HBase and Schema Design</a>.</p>
</dd>
<dt class="hdlist1">How can I store (fill in the blank) in HBase?</dt>
<dd>
<p>See <a href="#supported.datatypes">Supported Datatypes</a>.</p>
</dd>
<dt class="hdlist1">How can I handle secondary indexes in HBase?</dt>
<dd>
<p>See <a href="#secondary.indexes">Secondary Indexes and Alternate Query Paths</a>.</p>
</dd>
<dt class="hdlist1">Can I change a table&#8217;s rowkeys?</dt>
<dd>
<p>This is a very common question. You can&#8217;t. See <a href="#changing.rowkeys">Immutability of Rowkeys</a>.</p>
</dd>
<dt class="hdlist1">What APIs does HBase support?</dt>
<dd>
<p>See <a href="#datamodel">Data Model</a>, <a href="#architecture.client">Client</a>, and <a href="#external_apis">Apache HBase External APIs</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_mapreduce"><a class="anchor" href="#_mapreduce"></a>B.6. MapReduce</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How can I use MapReduce with HBase?</dt>
<dd>
<p>See <a href="#mapreduce">HBase and MapReduce</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_performance_and_troubleshooting"><a class="anchor" href="#_performance_and_troubleshooting"></a>B.7. Performance and Troubleshooting</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How can I improve HBase cluster performance?</dt>
<dd>
<p>See <a href="#performance">Apache HBase Performance Tuning</a>.</p>
</dd>
<dt class="hdlist1">How can I troubleshoot my HBase cluster?</dt>
<dd>
<p>See <a href="#trouble">Troubleshooting and Debugging Apache HBase</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_amazon_ec2"><a class="anchor" href="#_amazon_ec2"></a>B.8. Amazon EC2</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">I am running HBase on Amazon EC2 and&#8230;&#8203;</dt>
<dd>
<p>EC2 issues are a special case. See <a href="#trouble.ec2">Amazon EC2</a> and <a href="#perf.ec2">Amazon EC2</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_operations"><a class="anchor" href="#_operations"></a>B.9. Operations</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">How do I manage my HBase cluster?</dt>
<dd>
<p>See <a href="#ops_mgt">Apache HBase Operational Management</a>.</p>
</dd>
<dt class="hdlist1">How do I back up my HBase cluster?</dt>
<dd>
<p>See <a href="#ops.backup">HBase Backup</a>.</p>
</dd>
</dl>
</div>
</div>
<div class="sect2">
<h3 id="_hbase_in_action"><a class="anchor" href="#_hbase_in_action"></a>B.10. HBase in Action</h3>
<div class="dlist">
<dl>
<dt class="hdlist1">Where can I find interesting videos and presentations on HBase?</dt>
<dd>
<p>See <a href="#other.info">Other Information About HBase</a>.</p>
</dd>
</dl>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="appendix_acl_matrix"><a class="anchor" href="#appendix_acl_matrix"></a>Appendix C: Access Control Matrix</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The following matrix shows the permission set required to perform operations in HBase.
Before using the table, read through the information about how to interpret it.</p>
</div>
<div class="paragraph">
<div class="title">Interpreting the ACL Matrix Table</div>
<p>The following conventions are used in the ACL Matrix table:</p>
</div>
<div class="sect2">
<h3 id="_scopes"><a class="anchor" href="#_scopes"></a>C.1. Scopes</h3>
<div class="paragraph">
<p>Permissions are evaluated starting at the widest scope and working to the narrowest scope.</p>
</div>
<div class="paragraph">
<p>A scope corresponds to a level of the data model. From broadest to narrowest, the scopes are as follows:</p>
</div>
<div class="ulist">
<div class="title">Scopes</div>
<ul>
<li>
<p>Global</p>
</li>
<li>
<p>Namespace (NS)</p>
</li>
<li>
<p>Table</p>
</li>
<li>
<p>Column Family (CF)</p>
</li>
<li>
<p>Column Qualifier (CQ)</p>
</li>
<li>
<p>Cell</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>For instance, a permission granted at table level dominates any grants done at the Column Family, Column Qualifier, or cell level. The user can do what that grant implies at any location in the table. A permission granted at global scope dominates all: the user is always allowed to take that action everywhere.</p>
</div>
</div>
<div class="sect2">
<h3 id="_permissions"><a class="anchor" href="#_permissions"></a>C.2. Permissions</h3>
<div class="paragraph">
<p>Possible permissions include the following:</p>
</div>
<div class="ulist">
<div class="title">Permissions</div>
<ul>
<li>
<p>Superuser - a special user that belongs to group "supergroup" and has unlimited access</p>
</li>
<li>
<p>Admin (A)</p>
</li>
<li>
<p>Create (C)</p>
</li>
<li>
<p>Write (W)</p>
</li>
<li>
<p>Read (R)</p>
</li>
<li>
<p>Execute (X)</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>For the most part, permissions work in an expected way, with the following caveats:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Having Write permission does not imply Read permission.</dt>
<dd>
<p>It is possible and sometimes desirable for a user to be able to write data that same user cannot read. One such example is a log-writing process.</p>
</dd>
<dt class="hdlist1">The <span class="systemitem">hbase:meta</span> table is readable by every user, regardless of the user&#8217;s other grants or restrictions.</dt>
<dd>
<p>This is a requirement for HBase to function correctly.</p>
</dd>
<dt class="hdlist1"><code>CheckAndPut</code> and <code>CheckAndDelete</code> operations will fail if the user does not have both Write and Read permission.</dt>
<dt class="hdlist1"><code>Increment</code> and <code>Append</code> operations do not require Read access.</dt>
<dt class="hdlist1">The <code>superuser</code>, as the name suggests has permissions to perform all possible operations.</dt>
<dt class="hdlist1">And for the operations marked with *, the checks are done in post hook and only subset of results satisfying access checks are returned back to the user.</dt>
<dd>
<p>The following table is sorted by the interface that provides each operation.
In case the table goes out of date, the unit tests which check for accuracy of permissions can be found in <em>hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java</em>, and the access controls themselves can be examined in <em>hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java</em>.</p>
</dd>
</dl>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 24. ACL Matrix</caption>
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Interface</th>
<th class="tableblock halign-left valign-top">Operation</th>
<th class="tableblock halign-left valign-top">Permissions</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Master</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">createTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(C)|NS(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">modifyTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">deleteTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">truncateTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">addColumn</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">modifyColumn</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)|column(A)|column(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">deleteColumn</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)|column(A)|column(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">enableTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">disableTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">disableAclTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Not allowed</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">move</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">assign</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">unassign</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">regionOffline</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">balance</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">balanceSwitch</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">shutdown</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">stopMaster</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">snapshot</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">listSnapshot</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|SnapshotOwner</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">cloneSnapshot</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|(SnapshotOwner &amp; TableName matches)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">restoreSnapshot</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|SnapshotOwner &amp; (NS(A)|TableOwner|table(A))</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">deleteSnapshot</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|SnapshotOwner</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">createNamespace</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">deleteNamespace</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">modifyNamespace</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getNamespaceDescriptor</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">listNamespaceDescriptors*</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">flushTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getTableDescriptors*</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|NS(A)|NS(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getTableNames*</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|TableOwner|Any global or table perm</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">setUserQuota(global level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">setUserQuota(namespace level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">setUserQuota(Table level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">setTableQuota</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|NS(A)|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">setNamespaceQuota</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">addReplicationPeer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">removeReplicationPeer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">enableReplicationPeer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">disableReplicationPeer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getReplicationPeerConfig</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">updateReplicationPeerConfig</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">listReplicationPeers</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getClusterStatus</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">any user</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Region</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">openRegion</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">closeRegion</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">flush</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">split</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|TableOwner|TableOwner|table(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">compact</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)|global(C)|TableOwner|table(A)|table(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getClosestRowBefore</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getOp</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">exists</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">put</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|table(W)|TableOwner|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">delete</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|table(W)|TableOwner|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">batchMutate</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">checkAndPut</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(RW)|NS(RW)|TableOwner|table(RW)|CF(RW)|CQ(RW)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">checkAndPutAfterRowLock</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|Table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">checkAndDelete</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(RW)|NS(RW)|TableOwner|table(RW)|CF(RW)|CQ(RW)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">checkAndDeleteAfterRowLock</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">incrementColumnValue</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">append</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">appendAfterRowLock</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">increment</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">incrementAfterRowLock</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)|NS(W)|TableOwner|table(W)|CF(W)|CQ(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">scannerOpen</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">scannerNext</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">scannerClose</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(R)|NS(R)|TableOwner|table(R)|CF(R)|CQ(R)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">bulkLoadHFile</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(C)|TableOwner|table(C)|CF(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">prepareBulkLoad</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(C)|TableOwner|table(C)|CF(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">cleanupBulkLoad</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(C)|TableOwner|table(C)|CF(C)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Endpoint</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">invoke</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(X)|NS(X)|TableOwner|table(X)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">AccessController</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">grant(global level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">grant(namespace level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">grant(table level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)|TableOwner|table(A)|CF(A)|CQ(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">revoke(global level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">revoke(namespace level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">revoke(table level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)|TableOwner|table(A)|CF(A)|CQ(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getUserPermissions(global level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getUserPermissions(namespace level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getUserPermissions(table level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|NS(A)|TableOwner|table(A)|CF(A)|CQ(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">hasPermission(table level)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">global(A)|SelfUserCheck</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">RegionServer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">stopRegionServer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">mergeRegions</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">rollWALWriterRequest</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">replicateLogEntries</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(W)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">RSGroup</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">addRSGroup</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">balanceRSGroup</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getRSGroupInfo</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getRSGroupInfoOfTable</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">getRSGroupOfServer</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">listRSGroups</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">moveServers</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">moveServersAndTables</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">moveTables</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">removeRSGroup</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">removeServers</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">superuser|global(A)</p></td>
</tr>
</tbody>
</table>
</div>
</div>
</div>
<div class="sect1">
<h2 id="compression"><a class="anchor" href="#compression"></a>Appendix D: Compression and Data Block Encoding In HBase</h2>
<div class="sectionbody">
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Codecs mentioned in this section are for encoding and decoding data blocks or row keys.
For information about replication codecs, see <a href="#cluster.replication.preserving.tags">cluster.replication.preserving.tags</a>.
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Some of the information in this section is pulled from a <a href="http://search-hadoop.com/m/lL12B1PFVhp1/v=threaded">discussion</a> on the HBase Development mailing list.</p>
</div>
<div class="paragraph">
<p>HBase supports several different compression algorithms which can be enabled on a ColumnFamily.
Data block encoding attempts to limit duplication of information in keys, taking advantage of some of the fundamental designs and patterns of HBase, such as sorted row keys and the schema of a given table.
Compressors reduce the size of large, opaque byte arrays in cells, and can significantly reduce the storage space needed to store uncompressed data.</p>
</div>
<div class="paragraph">
<p>Compressors and data block encoding can be used together on the same ColumnFamily.</p>
</div>
<div class="paragraph">
<div class="title">Changes Take Effect Upon Compaction</div>
<p>If you change compression or encoding for a ColumnFamily, the changes take effect during compaction.</p>
</div>
<div class="paragraph">
<p>Some codecs take advantage of capabilities built into Java, such as GZip compression. Others rely on native libraries. Native libraries may be available as part of Hadoop, such as LZ4. In this case, HBase only needs access to the appropriate shared library.</p>
</div>
<div class="paragraph">
<p>Other codecs, such as Google Snappy, need to be installed first.
Some codecs are licensed in ways that conflict with HBase&#8217;s license and cannot be shipped as part of HBase.</p>
</div>
<div class="paragraph">
<p>This section discusses common codecs that are used and tested with HBase.
No matter what codec you use, be sure to test that it is installed correctly and is available on all nodes in your cluster.
Extra operational steps may be necessary to be sure that codecs are available on newly-deployed nodes.
You can use the <a href="#compression.test">compression.test</a> utility to check that a given codec is correctly installed.</p>
</div>
<div class="paragraph">
<p>To configure HBase to use a compressor, see <a href="#compressor.install">compressor.install</a>.
To enable a compressor for a ColumnFamily, see <a href="#changing.compression">changing.compression</a>.
To enable data block encoding for a ColumnFamily, see <a href="#data.block.encoding.enable">data.block.encoding.enable</a>.</p>
</div>
<div class="ulist">
<div class="title">Block Compressors</div>
<ul>
<li>
<p>none</p>
</li>
<li>
<p>Snappy</p>
</li>
<li>
<p>LZO</p>
</li>
<li>
<p>LZ4</p>
</li>
<li>
<p>GZ</p>
</li>
</ul>
</div>
<div class="dlist">
<div class="title">Data Block Encoding Types</div>
<dl>
<dt class="hdlist1">Prefix</dt>
<dd>
<p> Often, keys are very similar. Specifically, keys often share a common prefix and only differ near the end. For instance, one key might be <code>RowKey:Family:Qualifier0</code> and the next key might be <code>RowKey:Family:Qualifier1</code>.
<br>
In Prefix encoding, an extra column is added which holds the length of the prefix shared between the current key and the previous key.
Assuming the first key here is totally different from the key before, its prefix length is 0.</p>
<div class="paragraph">
<p>The second key&#8217;s prefix length is <code>23</code>, since they have the first 23 characters in common.</p>
</div>
<div class="paragraph">
<p>Obviously if the keys tend to have nothing in common, Prefix will not provide much benefit.</p>
</div>
<div class="paragraph">
<p>The following image shows a hypothetical ColumnFamily with no data block encoding.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/data_block_no_encoding.png" alt="data block no encoding">
</div>
<div class="title">Figure 19. ColumnFamily with No Encoding</div>
</div>
<div class="paragraph">
<p>Here is the same data with prefix data encoding.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/data_block_prefix_encoding.png" alt="data block prefix encoding">
</div>
<div class="title">Figure 20. ColumnFamily with Prefix Encoding</div>
</div>
</dd>
<dt class="hdlist1">Diff</dt>
<dd>
<p>Diff encoding expands upon Prefix encoding.
Instead of considering the key sequentially as a monolithic series of bytes, each key field is split so that each part of the key can be compressed more efficiently.</p>
<div class="paragraph">
<p>Two new fields are added: timestamp and type.</p>
</div>
<div class="paragraph">
<p>If the ColumnFamily is the same as the previous row, it is omitted from the current row.</p>
</div>
<div class="paragraph">
<p>If the key length, value length or type are the same as the previous row, the field is omitted.</p>
</div>
<div class="paragraph">
<p>In addition, for increased compression, the timestamp is stored as a Diff from the previous row&#8217;s timestamp, rather than being stored in full.
Given the two row keys in the Prefix example, and given an exact match on timestamp and the same type, neither the value length, or type needs to be stored for the second row, and the timestamp value for the second row is just 0, rather than a full timestamp.</p>
</div>
<div class="paragraph">
<p>Diff encoding is disabled by default because writing and scanning are slower but more data is cached.</p>
</div>
<div class="paragraph">
<p>This image shows the same ColumnFamily from the previous images, with Diff encoding.</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/data_block_diff_encoding.png" alt="data block diff encoding">
</div>
<div class="title">Figure 21. ColumnFamily with Diff Encoding</div>
</div>
</dd>
<dt class="hdlist1">Fast Diff</dt>
<dd>
<p>Fast Diff works similar to Diff, but uses a faster implementation. It also adds another field which stores a single bit to track whether the data itself is the same as the previous row. If it is, the data is not stored again.</p>
<div class="paragraph">
<p>Fast Diff is the recommended codec to use if you have long keys or many columns.</p>
</div>
<div class="paragraph">
<p>The data format is nearly identical to Diff encoding, so there is not an image to illustrate it.</p>
</div>
</dd>
<dt class="hdlist1">Prefix Tree</dt>
<dd>
<p>Prefix tree encoding was introduced as an experimental feature in HBase 0.96.
It provides similar memory savings to the Prefix, Diff, and Fast Diff encoder, but provides faster random access at a cost of slower encoding speed.
It was removed in hbase-2.0.0. It was a good idea but little uptake. If interested in reviving this effort, write the hbase dev list.</p>
</dd>
</dl>
</div>
<div class="sect2">
<h3 id="data.block.encoding.types"><a class="anchor" href="#data.block.encoding.types"></a>D.1. Which Compressor or Data Block Encoder To Use</h3>
<div class="paragraph">
<p>The compression or codec type to use depends on the characteristics of your data. Choosing the wrong type could cause your data to take more space rather than less, and can have performance implications.</p>
</div>
<div class="paragraph">
<p>In general, you need to weigh your options between smaller size and faster compression/decompression. Following are some general guidelines, expanded from a discussion at <a href="http://search-hadoop.com/m/lL12B1PFVhp1">Documenting Guidance on compression and codecs</a>.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>If you have long keys (compared to the values) or many columns, use a prefix encoder.
FAST_DIFF is recommended.</p>
</li>
<li>
<p>If the values are large (and not precompressed, such as images), use a data block compressor.</p>
</li>
<li>
<p>Use GZIP for <em class="firstterm">cold data</em>, which is accessed infrequently.
GZIP compression uses more CPU resources than Snappy or LZO, but provides a higher compression ratio.</p>
</li>
<li>
<p>Use Snappy or LZO for <em class="firstterm">hot data</em>, which is accessed frequently.
Snappy and LZO use fewer CPU resources than GZIP, but do not provide as high of a compression ratio.</p>
</li>
<li>
<p>In most cases, enabling Snappy or LZO by default is a good choice, because they have a low performance overhead and provide space savings.</p>
</li>
<li>
<p>Before Snappy became available by Google in 2011, LZO was the default.
Snappy has similar qualities as LZO but has been shown to perform better.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="hadoop.native.lib"><a class="anchor" href="#hadoop.native.lib"></a>D.2. Making use of Hadoop Native Libraries in HBase</h3>
<div class="paragraph">
<p>The Hadoop shared library has a bunch of facility including compression libraries and fast crc&#8217;ing&#8201;&#8212;&#8201;hardware crc&#8217;ing if your chipset supports it.
To make this facility available to HBase, do the following. HBase/Hadoop will fall back to use alternatives if it cannot find the native library
versions&#8201;&#8212;&#8201;or fail outright if you asking for an explicit compressor and there is no alternative available.</p>
</div>
<div class="paragraph">
<p>First make sure of your Hadoop. Fix this message if you are seeing it starting Hadoop processes:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>16/02/09 22:40:24 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable</pre>
</div>
</div>
<div class="paragraph">
<p>It means is not properly pointing at its native libraries or the native libs were compiled for another platform.
Fix this first.</p>
</div>
<div class="paragraph">
<p>Then if you see the following in your HBase logs, you know that HBase was unable to locate the Hadoop native libraries:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="integer">2014</span>-<span class="integer">08</span>-<span class="octal">07</span> <span class="integer">09</span>:<span class="integer">26</span>:<span class="integer">20</span>,<span class="integer">139</span> WARN [main] util.NativeCodeLoader: Unable to load <span class="directive">native</span>-hadoop library <span class="keyword">for</span> your platform... using builtin-java classes where applicable</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the libraries loaded successfully, the WARN message does not show. Usually this means you are good to go but read on.</p>
</div>
<div class="paragraph">
<p>Let&#8217;s presume your Hadoop shipped with a native library that suits the platform you are running HBase on.
To check if the Hadoop native library is available to HBase, run the following tool (available in Hadoop 2.1 and greater):</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> ./bin/hbase --config ~/conf_hbase org.apache.hadoop.util.NativeLibraryChecker
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="integer">26</span> <span class="integer">13</span>:<span class="integer">15</span>:<span class="integer">38</span>,<span class="integer">717</span> WARN [main] util.NativeCodeLoader: Unable to load <span class="directive">native</span>-hadoop library <span class="keyword">for</span> your platform... using builtin-java classes where applicable
Native library checking:
hadoop: <span class="predefined-constant">false</span>
zlib: <span class="predefined-constant">false</span>
snappy: <span class="predefined-constant">false</span>
lz4: <span class="predefined-constant">false</span>
bzip2: <span class="predefined-constant">false</span>
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="integer">26</span> <span class="integer">13</span>:<span class="integer">15</span>:<span class="integer">38</span>,<span class="integer">863</span> INFO [main] util.ExitUtil: Exiting with status <span class="integer">1</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Above shows that the native hadoop library is not available in HBase context.</p>
</div>
<div class="paragraph">
<p>The above NativeLibraryChecker tool may come back saying all is hunky-dory&#8201;&#8212;&#8201;i.e. all libs show 'true', that they are available&#8201;&#8212;&#8201;but follow the below
presecription anyways to ensure the native libs are available in HBase context,
when it goes to use them.</p>
</div>
<div class="paragraph">
<p>To fix the above, either copy the Hadoop native libraries local or symlink to them if the Hadoop and HBase stalls are adjacent in the filesystem.
You could also point at their location by setting the <code>LD_LIBRARY_PATH</code> environment variable in your hbase-env.sh.</p>
</div>
<div class="paragraph">
<p>Where the JVM looks to find native libraries is "system dependent" (See <code>java.lang.System#loadLibrary(name)</code>). On linux, by default, is going to look in <em>lib/native/PLATFORM</em> where <code>PLATFORM</code> is the label for the platform your HBase is installed on.
On a local linux machine, it seems to be the concatenation of the java properties <code>os.name</code> and <code>os.arch</code> followed by whether 32 or 64 bit.
HBase on startup prints out all of the java system properties so find the os.name and os.arch in the log.
For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">...
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="octal">06</span> <span class="integer">15</span>:<span class="integer">27</span>:<span class="integer">22</span>,<span class="integer">853</span> INFO [main] zookeeper.ZooKeeper: Client environment:os.name=Linux
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="octal">06</span> <span class="integer">15</span>:<span class="integer">27</span>:<span class="integer">22</span>,<span class="integer">853</span> INFO [main] zookeeper.ZooKeeper: Client environment:os.arch=amd64
...</code></pre>
</div>
</div>
<div class="paragraph">
<p>So in this case, the PLATFORM string is <code>Linux-amd64-64</code>.
Copying the Hadoop native libraries or symlinking at <em>lib/native/Linux-amd64-64</em> will ensure they are found.
Rolling restart after you have made this change.</p>
</div>
<div class="paragraph">
<p>Here is an example of how you would set up the symlinks.
Let the hadoop and hbase installs be in your home directory. Assume your hadoop native libs
are at ~/hadoop/lib/native. Assume you are on a Linux-amd64-64 platform. In this case,
you would do the following to link the hadoop native lib so hbase could find them.</p>
</div>
<div class="listingblock">
<div class="content">
<pre>...
$ mkdir -p ~/hbaseLinux-amd64-64 -&gt; /home/stack/hadoop/lib/native/lib/native/
$ cd ~/hbase/lib/native/
$ ln -s ~/hadoop/lib/native Linux-amd64-64
$ ls -la
# Linux-amd64-64 -&gt; /home/USER/hadoop/lib/native
...</pre>
</div>
</div>
<div class="paragraph">
<p>If you see PureJavaCrc32C in a stack track or if you see something like the below in a perf trace, then native is not working; you are using the java CRC functions rather than native:</p>
</div>
<div class="listingblock">
<div class="content">
<pre> 5.02% perf-53601.map [.] Lorg/apache/hadoop/util/PureJavaCrc32C;.update</pre>
</div>
</div>
<div class="paragraph">
<p>See <a href="https://issues.apache.org/jira/browse/HBASE-11927">HBASE-11927 Use Native Hadoop Library for HFile checksum (And flip default from CRC32 to CRC32C)</a>,
for more on native checksumming support. See in particular the release note for how to check if your hardware to see if your processor has support for hardware CRCs.
Or checkout the Apache <a href="https://blogs.apache.org/hbase/entry/saving_cpu_using_native_hadoop">Checksums in HBase</a> blog post.</p>
</div>
<div class="paragraph">
<p>Here is example of how to point at the Hadoop libs with <code>LD_LIBRARY_PATH</code> environment variable:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="error">$</span> LD_LIBRARY_PATH=~/hadoop-<span class="float">2.5</span><span class="float">.0</span>-SNAPSHOT/lib/<span class="directive">native</span> ./bin/hbase --config ~/conf_hbase org.apache.hadoop.util.NativeLibraryChecker
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="integer">26</span> <span class="integer">13</span>:<span class="integer">42</span>:<span class="integer">49</span>,<span class="integer">332</span> INFO [main] bzip2.Bzip2Factory: Successfully loaded &amp; initialized <span class="directive">native</span>-bzip2 library system-<span class="directive">native</span>
<span class="integer">2014</span>-<span class="integer">08</span>-<span class="integer">26</span> <span class="integer">13</span>:<span class="integer">42</span>:<span class="integer">49</span>,<span class="integer">337</span> INFO [main] zlib.ZlibFactory: Successfully loaded &amp; initialized <span class="directive">native</span>-zlib library
Native library checking:
hadoop: <span class="predefined-constant">true</span> /home/stack/hadoop-<span class="float">2.5</span><span class="float">.0</span>-SNAPSHOT/lib/<span class="directive">native</span>/libhadoop.so<span class="float">.1</span><span class="float">.0</span><span class="float">.0</span>
zlib: <span class="predefined-constant">true</span> /lib64/libz.so<span class="float">.1</span>
snappy: <span class="predefined-constant">true</span> /usr/lib64/libsnappy.so<span class="float">.1</span>
lz4: <span class="predefined-constant">true</span> revision:<span class="integer">99</span>
bzip2: <span class="predefined-constant">true</span> /lib64/libbz2.so<span class="float">.1</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>Set in <em>hbase-env.sh</em> the LD_LIBRARY_PATH environment variable when starting your HBase.</p>
</div>
</div>
<div class="sect2">
<h3 id="_compressor_configuration_installation_and_use"><a class="anchor" href="#_compressor_configuration_installation_and_use"></a>D.3. Compressor Configuration, Installation, and Use</h3>
<div class="sect3">
<h4 id="compressor.install"><a class="anchor" href="#compressor.install"></a>D.3.1. Configure HBase For Compressors</h4>
<div class="paragraph">
<p>Before HBase can use a given compressor, its libraries need to be available.
Due to licensing issues, only GZ compression is available to HBase (via native Java libraries) in a default installation.
Other compression libraries are available via the shared library bundled with your hadoop.
The hadoop native library needs to be findable when HBase starts.
See</p>
</div>
<div class="paragraph">
<div class="title">Compressor Support On the Master</div>
<p>A new configuration setting was introduced in HBase 0.95, to check the Master to determine which data block encoders are installed and configured on it, and assume that the entire cluster is configured the same.
This option, <code>hbase.master.check.compression</code>, defaults to <code>true</code>.
This prevents the situation described in <a href="https://issues.apache.org/jira/browse/HBASE-6370">HBASE-6370</a>, where a table is created or modified to support a codec that a region server does not support, leading to failures that take a long time to occur and are difficult to debug.</p>
</div>
<div class="paragraph">
<p>If <code>hbase.master.check.compression</code> is enabled, libraries for all desired compressors need to be installed and configured on the Master, even if the Master does not run a region server.</p>
</div>
<div class="paragraph">
<div class="title">Install GZ Support Via Native Libraries</div>
<p>HBase uses Java&#8217;s built-in GZip support unless the native Hadoop libraries are available on the CLASSPATH.
The recommended way to add libraries to the CLASSPATH is to set the environment variable <code>HBASE_LIBRARY_PATH</code> for the user running HBase.
If native libraries are not available and Java&#8217;s GZIP is used, <code>Got brand-new compressor</code> reports will be present in the logs.
See <a href="#brand.new.compressor">brand.new.compressor</a>).</p>
</div>
<div id="lzo.compression" class="paragraph">
<div class="title">Install LZO Support</div>
<p>HBase cannot ship with LZO because of incompatibility between HBase, which uses an Apache Software License (ASL) and LZO, which uses a GPL license.
See the <a href="https://github.com/twitter/hadoop-lzo/blob/master/README.md">Hadoop-LZO at Twitter</a> for information on configuring LZO support for HBase.</p>
</div>
<div class="paragraph">
<p>If you depend upon LZO compression, consider configuring your RegionServers to fail to start if LZO is not available.
See <a href="#hbase.regionserver.codecs">hbase.regionserver.codecs</a>.</p>
</div>
<div id="lz4.compression" class="paragraph">
<div class="title">Configure LZ4 Support</div>
<p>LZ4 support is bundled with Hadoop.
Make sure the hadoop shared library (libhadoop.so) is accessible when you start HBase.
After configuring your platform (see <a href="#hadoop.native.lib">hadoop.native.lib</a>), you can make a symbolic link from HBase to the native Hadoop libraries.
This assumes the two software installs are colocated.
For example, if my 'platform' is Linux-amd64-64:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="bourne">$ cd $HBASE_HOME
$ mkdir lib/native
$ ln -s $HADOOP_HOME/lib/native lib/native/Linux-amd64-64</code></pre>
</div>
</div>
<div class="paragraph">
<p>Use the compression tool to check that LZ4 is installed on all nodes.
Start up (or restart) HBase.
Afterward, you can create and alter tables to enable LZ4 as a compression codec.:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>hbase(main):003:0&gt; alter 'TestTable', {NAME =&gt; 'info', COMPRESSION =&gt; 'LZ4'}</pre>
</div>
</div>
<div id="snappy.compression.installation" class="paragraph">
<div class="title">Install Snappy Support</div>
<p>HBase does not ship with Snappy support because of licensing issues.
You can install Snappy binaries (for instance, by using yum install snappy on CentOS) or build Snappy from source.
After installing Snappy, search for the shared library, which will be called <em>libsnappy.so.X</em> where X is a number.
If you built from source, copy the shared library to a known location on your system, such as <em>/opt/snappy/lib/</em>.</p>
</div>
<div class="paragraph">
<p>In addition to the Snappy library, HBase also needs access to the Hadoop shared library, which will be called something like <em>libhadoop.so.X.Y</em>, where X and Y are both numbers.
Make note of the location of the Hadoop library, or copy it to the same location as the Snappy library.</p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
<div class="paragraph">
<p>The Snappy and Hadoop libraries need to be available on each node of your cluster.
See <a href="#compression.test">compression.test</a> to find out how to test that this is the case.</p>
</div>
<div class="paragraph">
<p>See <a href="#hbase.regionserver.codecs">hbase.regionserver.codecs</a> to configure your RegionServers to fail to start if a given compressor is not available.</p>
</div>
</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>Each of these library locations need to be added to the environment variable <code>HBASE_LIBRARY_PATH</code> for the operating system user that runs HBase.
You need to restart the RegionServer for the changes to take effect.</p>
</div>
<div id="compression.test" class="paragraph">
<div class="title">CompressionTest</div>
<p>You can use the CompressionTest tool to verify that your compressor is available to HBase:</p>
</div>
<div class="listingblock">
<div class="content">
<pre> $ hbase org.apache.hadoop.hbase.util.CompressionTest hdfs://host/path/to/hbase snappy</pre>
</div>
</div>
<div id="hbase.regionserver.codecs" class="paragraph">
<div class="title">Enforce Compression Settings On a RegionServer</div>
<p>You can configure a RegionServer so that it will fail to restart if compression is configured incorrectly, by adding the option hbase.regionserver.codecs to the <em>hbase-site.xml</em>, and setting its value to a comma-separated list of codecs that need to be available.
For example, if you set this property to <code>lzo,gz</code>, the RegionServer would fail to start if both compressors were not available.
This would prevent a new server from being added to the cluster without having codecs configured properly.</p>
</div>
</div>
<div class="sect3">
<h4 id="changing.compression"><a class="anchor" href="#changing.compression"></a>D.3.2. Enable Compression On a ColumnFamily</h4>
<div class="paragraph">
<p>To enable compression for a ColumnFamily, use an <code>alter</code> command.
You do not need to re-create the table or copy data.
If you are changing codecs, be sure the old codec is still available until all the old StoreFiles have been compacted.</p>
</div>
<div class="listingblock">
<div class="title">Enabling Compression on a ColumnFamily of an Existing Table using HBaseShell</div>
<div class="content">
<pre>hbase&gt; disable 'test'
hbase&gt; alter 'test', {NAME =&gt; 'cf', COMPRESSION =&gt; 'GZ'}
hbase&gt; enable 'test'</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Creating a New Table with Compression On a ColumnFamily</div>
<div class="content">
<pre>hbase&gt; create 'test2', { NAME =&gt; 'cf2', COMPRESSION =&gt; 'SNAPPY' }</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Verifying a ColumnFamily&#8217;s Compression Settings</div>
<div class="content">
<pre>hbase&gt; describe 'test'
DESCRIPTION ENABLED
'test', {NAME =&gt; 'cf', DATA_BLOCK_ENCODING =&gt; 'NONE false
', BLOOMFILTER =&gt; 'ROW', REPLICATION_SCOPE =&gt; '0',
VERSIONS =&gt; '1', COMPRESSION =&gt; 'GZ', MIN_VERSIONS
=&gt; '0', TTL =&gt; 'FOREVER', KEEP_DELETED_CELLS =&gt; 'fa
lse', BLOCKSIZE =&gt; '65536', IN_MEMORY =&gt; 'false', B
LOCKCACHE =&gt; 'true'}
1 row(s) in 0.1070 seconds</pre>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_testing_compression_performance"><a class="anchor" href="#_testing_compression_performance"></a>D.3.3. Testing Compression Performance</h4>
<div class="paragraph">
<p>HBase includes a tool called LoadTestTool which provides mechanisms to test your compression performance.
You must specify either <code>-write</code> or <code>-update-read</code> as your first parameter, and if you do not specify another parameter, usage advice is printed for each option.</p>
</div>
<div class="listingblock">
<div class="title">LoadTestTool Usage</div>
<div class="content">
<pre>$ bin/hbase org.apache.hadoop.hbase.util.LoadTestTool -h
usage: bin/hbase org.apache.hadoop.hbase.util.LoadTestTool &lt;options&gt;
Options:
-batchupdate Whether to use batch as opposed to separate
updates for every column in a row
-bloom &lt;arg&gt; Bloom filter type, one of [NONE, ROW, ROWCOL]
-compression &lt;arg&gt; Compression type, one of [LZO, GZ, NONE, SNAPPY,
LZ4]
-data_block_encoding &lt;arg&gt; Encoding algorithm (e.g. prefix compression) to
use for data blocks in the test column family, one
of [NONE, PREFIX, DIFF, FAST_DIFF, ROW_INDEX_V1].
-encryption &lt;arg&gt; Enables transparent encryption on the test table,
one of [AES]
-generator &lt;arg&gt; The class which generates load for the tool. Any
args for this class can be passed as colon
separated after class name
-h,--help Show usage
-in_memory Tries to keep the HFiles of the CF inmemory as far
as possible. Not guaranteed that reads are always
served from inmemory
-init_only Initialize the test table only, don't do any
loading
-key_window &lt;arg&gt; The 'key window' to maintain between reads and
writes for concurrent write/read workload. The
default is 0.
-max_read_errors &lt;arg&gt; The maximum number of read errors to tolerate
before terminating all reader threads. The default
is 10.
-multiput Whether to use multi-puts as opposed to separate
puts for every column in a row
-num_keys &lt;arg&gt; The number of keys to read/write
-num_tables &lt;arg&gt; A positive integer number. When a number n is
speicfied, load test tool will load n table
parallely. -tn parameter value becomes table name
prefix. Each table name is in format
&lt;tn&gt;_1...&lt;tn&gt;_n
-read &lt;arg&gt; &lt;verify_percent&gt;[:&lt;#threads=20&gt;]
-regions_per_server &lt;arg&gt; A positive integer number. When a number n is
specified, load test tool will create the test
table with n regions per server
-skip_init Skip the initialization; assume test table already
exists
-start_key &lt;arg&gt; The first key to read/write (a 0-based index). The
default value is 0.
-tn &lt;arg&gt; The name of the table to read or write
-update &lt;arg&gt; &lt;update_percent&gt;[:&lt;#threads=20&gt;][:&lt;#whether to
ignore nonce collisions=0&gt;]
-write &lt;arg&gt; &lt;avg_cols_per_key&gt;:&lt;avg_data_size&gt;[:&lt;#threads=20&gt;]
-zk &lt;arg&gt; ZK quorum as comma-separated host names without
port numbers
-zk_root &lt;arg&gt; name of parent znode in zookeeper</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Example Usage of LoadTestTool</div>
<div class="content">
<pre>$ hbase org.apache.hadoop.hbase.util.LoadTestTool -write 1:10:100 -num_keys 1000000
-read 100:30 -num_tables 1 -data_block_encoding NONE -tn load_test_tool_NONE</pre>
</div>
</div>
</div>
</div>
<div class="sect2">
<h3 id="data.block.encoding.enable"><a class="anchor" href="#data.block.encoding.enable"></a>D.4. Enable Data Block Encoding</h3>
<div class="paragraph">
<p>Codecs are built into HBase so no extra configuration is needed.
Codecs are enabled on a table by setting the <code>DATA_BLOCK_ENCODING</code> property.
Disable the table before altering its DATA_BLOCK_ENCODING setting.
Following is an example using HBase Shell:</p>
</div>
<div class="listingblock">
<div class="title">Enable Data Block Encoding On a Table</div>
<div class="content">
<pre>hbase&gt; disable 'test'
hbase&gt; alter 'test', { NAME =&gt; 'cf', DATA_BLOCK_ENCODING =&gt; 'FAST_DIFF' }
Updating all regions with the new schema...
0/1 regions updated.
1/1 regions updated.
Done.
0 row(s) in 2.2820 seconds
hbase&gt; enable 'test'
0 row(s) in 0.1580 seconds</pre>
</div>
</div>
<div class="listingblock">
<div class="title">Verifying a ColumnFamily&#8217;s Data Block Encoding</div>
<div class="content">
<pre>hbase&gt; describe 'test'
DESCRIPTION ENABLED
'test', {NAME =&gt; 'cf', DATA_BLOCK_ENCODING =&gt; 'FAST true
_DIFF', BLOOMFILTER =&gt; 'ROW', REPLICATION_SCOPE =&gt;
'0', VERSIONS =&gt; '1', COMPRESSION =&gt; 'GZ', MIN_VERS
IONS =&gt; '0', TTL =&gt; 'FOREVER', KEEP_DELETED_CELLS =
&gt; 'false', BLOCKSIZE =&gt; '65536', IN_MEMORY =&gt; 'fals
e', BLOCKCACHE =&gt; 'true'}
1 row(s) in 0.0650 seconds</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="sql"><a class="anchor" href="#sql"></a>Appendix E: SQL over HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The following projects offer some support for SQL over HBase.</p>
</div>
<div class="sect2">
<h3 id="phoenix"><a class="anchor" href="#phoenix"></a>E.1. Apache Phoenix</h3>
<div class="paragraph">
<p><a href="https://phoenix.apache.org">Apache Phoenix</a></p>
</div>
</div>
<div class="sect2">
<h3 id="_trafodion"><a class="anchor" href="#_trafodion"></a>E.2. Trafodion</h3>
<div class="paragraph">
<p><a href="https://trafodion.incubator.apache.org/">Trafodion: Transactional SQL-on-HBase</a></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ycsb"><a class="anchor" href="#ycsb"></a>Appendix F: YCSB</h2>
<div class="sectionbody">
<div class="paragraph">
<p><a href="https://github.com/brianfrankcooper/YCSB/">YCSB: The
Yahoo! Cloud Serving Benchmark</a> and HBase</p>
</div>
<div class="paragraph">
<p>TODO: Describe how YCSB is poor for putting up a decent cluster load.</p>
</div>
<div class="paragraph">
<p>TODO: Describe setup of YCSB for HBase.
In particular, presplit your tables before you start a run.
See <a href="https://issues.apache.org/jira/browse/HBASE-4163">HBASE-4163 Create Split Strategy for YCSB Benchmark</a> for why and a little shell command for how to do it.</p>
</div>
<div class="paragraph">
<p>Ted Dunning redid YCSB so it&#8217;s mavenized and added facility for verifying workloads.
See <a href="https://github.com/tdunning/YCSB">Ted Dunning&#8217;s YCSB</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_hfile_format_2"><a class="anchor" href="#_hfile_format_2"></a>Appendix G: HFile format</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This appendix describes the evolution of the HFile format.</p>
</div>
<div class="sect2">
<h3 id="hfilev1"><a class="anchor" href="#hfilev1"></a>G.1. HBase File Format (version 1)</h3>
<div class="paragraph">
<p>As we will be discussing changes to the HFile format, it is useful to give a short overview of the original (HFile version 1) format.</p>
</div>
<div class="sect3">
<h4 id="hfilev1.overview"><a class="anchor" href="#hfilev1.overview"></a>G.1.1. Overview of Version 1</h4>
<div class="paragraph">
<p>An HFile in version 1 format is structured as follows:</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/hfile.png" alt="HFile Version 1">
</div>
<div class="title">Figure 22. HFile V1 Format</div>
</div>
</div>
<div class="sect3">
<h4 id="_block_index_format_in_version_1"><a class="anchor" href="#_block_index_format_in_version_1"></a>G.1.2. Block index format in version 1</h4>
<div class="paragraph">
<p>The block index in version 1 is very straightforward.
For each entry, it contains:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Offset (long)</p>
</li>
<li>
<p>Uncompressed size (int)</p>
</li>
<li>
<p>Key (a serialized byte array written using Bytes.writeByteArray)</p>
<div class="olist loweralpha">
<ol class="loweralpha" type="a">
<li>
<p>Key length as a variable-length integer (VInt)</p>
</li>
<li>
<p>Key bytes</p>
</li>
</ol>
</div>
</li>
</ol>
</div>
<div class="paragraph">
<p>The number of entries in the block index is stored in the fixed file trailer, and has to be passed in to the method that reads the block index.
One of the limitations of the block index in version 1 is that it does not provide the compressed size of a block, which turns out to be necessary for decompression.
Therefore, the HFile reader has to infer this compressed size from the offset difference between blocks.
We fix this limitation in version 2, where we store on-disk block size instead of uncompressed size, and get uncompressed size from the block header.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hfilev2"><a class="anchor" href="#hfilev2"></a>G.2. HBase file format with inline blocks (version 2)</h3>
<div class="paragraph">
<p>Note: this feature was introduced in HBase 0.92</p>
</div>
<div class="sect3">
<h4 id="_motivation"><a class="anchor" href="#_motivation"></a>G.2.1. Motivation</h4>
<div class="paragraph">
<p>We found it necessary to revise the HFile format after encountering high memory usage and slow startup times caused by large Bloom filters and block indexes in the region server.
Bloom filters can get as large as 100 MB per HFile, which adds up to 2 GB when aggregated over 20 regions.
Block indexes can grow as large as 6 GB in aggregate size over the same set of regions.
A region is not considered opened until all of its block index data is loaded.
Large Bloom filters produce a different performance problem: the first get request that requires a Bloom filter lookup will incur the latency of loading the entire Bloom filter bit array.</p>
</div>
<div class="paragraph">
<p>To speed up region server startup we break Bloom filters and block indexes into multiple blocks and write those blocks out as they fill up, which also reduces the HFile writer&#8217;s memory footprint.
In the Bloom filter case, "filling up a block" means accumulating enough keys to efficiently utilize a fixed-size bit array, and in the block index case we accumulate an "index block" of the desired size.
Bloom filter blocks and index blocks (we call these "inline blocks") become interspersed with data blocks, and as a side effect we can no longer rely on the difference between block offsets to determine data block length, as it was done in version 1.</p>
</div>
<div class="paragraph">
<p>HFile is a low-level file format by design, and it should not deal with application-specific details such as Bloom filters, which are handled at StoreFile level.
Therefore, we call Bloom filter blocks in an HFile "inline" blocks.
We also supply HFile with an interface to write those inline blocks.</p>
</div>
<div class="paragraph">
<p>Another format modification aimed at reducing the region server startup time is to use a contiguous "load-on-open" section that has to be loaded in memory at the time an HFile is being opened.
Currently, as an HFile opens, there are separate seek operations to read the trailer, data/meta indexes, and file info.
To read the Bloom filter, there are two more seek operations for its "data" and "meta" portions.
In version 2, we seek once to read the trailer and seek again to read everything else we need to open the file from a contiguous block.</p>
</div>
</div>
<div class="sect3">
<h4 id="hfilev2.overview"><a class="anchor" href="#hfilev2.overview"></a>G.2.2. Overview of Version 2</h4>
<div class="paragraph">
<p>The version of HBase introducing the above features reads both version 1 and 2 HFiles, but only writes version 2 HFiles.
A version 2 HFile is structured as follows:</p>
</div>
<div class="imageblock">
<div class="content">
<img src="images/hfilev2.png" alt="HFile Version 2">
</div>
<div class="title">Figure 23. HFile Version 2 Structure</div>
</div>
</div>
<div class="sect3">
<h4 id="_unified_version_2_block_format"><a class="anchor" href="#_unified_version_2_block_format"></a>G.2.3. Unified version 2 block format</h4>
<div class="paragraph">
<p>In the version 2 every block in the data section contains the following fields:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>8 bytes: Block type, a sequence of bytes equivalent to version 1&#8217;s "magic records". Supported block types are:</p>
<div class="olist loweralpha">
<ol class="loweralpha" type="a">
<li>
<p>DATA – data blocks</p>
</li>
<li>
<p>LEAF_INDEX – leaf-level index blocks in a multi-level-block-index</p>
</li>
<li>
<p>BLOOM_CHUNK – Bloom filter chunks</p>
</li>
<li>
<p>META – meta blocks (not used for Bloom filters in version 2 anymore)</p>
</li>
<li>
<p>INTERMEDIATE_INDEX – intermediate-level index blocks in a multi-level blockindex</p>
</li>
<li>
<p>ROOT_INDEX – root-level index blocks in a multi-level block index</p>
</li>
<li>
<p>FILE_INFO – the ''file info'' block, a small key-value map of metadata</p>
</li>
<li>
<p>BLOOM_META – a Bloom filter metadata block in the load-on-open section</p>
</li>
<li>
<p>TRAILER – a fixed-size file trailer.
As opposed to the above, this is not an HFile v2 block but a fixed-size (for each HFile version) data structure</p>
</li>
<li>
<p>INDEX_V1 – this block type is only used for legacy HFile v1 block</p>
</li>
</ol>
</div>
</li>
<li>
<p>Compressed size of the block&#8217;s data, not including the header (int).</p>
<div class="paragraph">
<p>Can be used for skipping the current data block when scanning HFile data.</p>
</div>
</li>
<li>
<p>Uncompressed size of the block&#8217;s data, not including the header (int)</p>
<div class="paragraph">
<p>This is equal to the compressed size if the compression algorithm is NONE</p>
</div>
</li>
<li>
<p>File offset of the previous block of the same type (long)</p>
<div class="paragraph">
<p>Can be used for seeking to the previous data/index block</p>
</div>
</li>
<li>
<p>Compressed data (or uncompressed data if the compression algorithm is NONE).</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The above format of blocks is used in the following HFile sections:</p>
</div>
<div class="dlist">
<dl>
<dt class="hdlist1">Scanned block section</dt>
<dd>
<p>The section is named so because it contains all data blocks that need to be read when an HFile is scanned sequentially.
Also contains Leaf index blocks and Bloom chunk blocks.</p>
</dd>
<dt class="hdlist1">Non-scanned block section</dt>
<dd>
<p>This section still contains unified-format v2 blocks but it does not have to be read when doing a sequential scan.
This section contains "meta" blocks and intermediate-level index blocks.</p>
</dd>
</dl>
</div>
<div class="paragraph">
<p>We are supporting "meta" blocks in version 2 the same way they were supported in version 1, even though we do not store Bloom filter data in these blocks anymore.</p>
</div>
</div>
<div class="sect3">
<h4 id="_block_index_in_version_2"><a class="anchor" href="#_block_index_in_version_2"></a>G.2.4. Block index in version 2</h4>
<div class="paragraph">
<p>There are three types of block indexes in HFile version 2, stored in two different formats (root and non-root):</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Data index&#8201;&#8212;&#8201;version 2 multi-level block index, consisting of:</p>
<div class="olist loweralpha">
<ol class="loweralpha" type="a">
<li>
<p>Version 2 root index, stored in the data block index section of the file</p>
</li>
<li>
<p>Optionally, version 2 intermediate levels, stored in the non-root format in the data index section of the file. Intermediate levels can only be present if leaf level blocks are present</p>
</li>
<li>
<p>Optionally, version 2 leaf levels, stored in the non-root format inline with data blocks</p>
</li>
</ol>
</div>
</li>
<li>
<p>Meta index&#8201;&#8212;&#8201;version 2 root index format only, stored in the meta index section of the file</p>
</li>
<li>
<p>Bloom index&#8201;&#8212;&#8201;version 2 root index format only, stored in the ''load-on-open'' section as part of Bloom filter metadata.</p>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_root_block_index_format_in_version_2"><a class="anchor" href="#_root_block_index_format_in_version_2"></a>G.2.5. Root block index format in version 2</h4>
<div class="paragraph">
<p>This format applies to:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Root level of the version 2 data index</p>
</li>
<li>
<p>Entire meta and Bloom indexes in version 2, which are always single-level.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>A version 2 root index block is a sequence of entries of the following format, similar to entries of a version 1 block index, but storing on-disk size instead of uncompressed size.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Offset (long)</p>
<div class="paragraph">
<p>This offset may point to a data block or to a deeper-level index block.</p>
</div>
</li>
<li>
<p>On-disk size (int)</p>
</li>
<li>
<p>Key (a serialized byte array stored using Bytes.writeByteArray)</p>
</li>
<li>
<p>Key (VInt)</p>
</li>
<li>
<p>Key bytes</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>A single-level version 2 block index consists of just a single root index block.
To read a root index block of version 2, one needs to know the number of entries.
For the data index and the meta index the number of entries is stored in the trailer, and for the Bloom index it is stored in the compound Bloom filter metadata.</p>
</div>
<div class="paragraph">
<p>For a multi-level block index we also store the following fields in the root index block in the load-on-open section of the HFile, in addition to the data structure described above:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Middle leaf index block offset</p>
</li>
<li>
<p>Middle leaf block on-disk size (meaning the leaf index block containing the reference to the ''middle'' data block of the file)</p>
</li>
<li>
<p>The index of the mid-key (defined below) in the middle leaf-level block.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>These additional fields are used to efficiently retrieve the mid-key of the HFile used in HFile splits, which we define as the first key of the block with a zero-based index of (n – 1) / 2, if the total number of blocks in the HFile is n.
This definition is consistent with how the mid-key was determined in HFile version 1, and is reasonable in general, because blocks are likely to be the same size on average, but we don&#8217;t have any estimates on individual key/value pair sizes.</p>
</div>
<div class="paragraph">
<p>When writing a version 2 HFile, the total number of data blocks pointed to by every leaf-level index block is kept track of.
When we finish writing and the total number of leaf-level blocks is determined, it is clear which leaf-level block contains the mid-key, and the fields listed above are computed.
When reading the HFile and the mid-key is requested, we retrieve the middle leaf index block (potentially from the block cache) and get the mid-key value from the appropriate position inside that leaf block.</p>
</div>
</div>
<div class="sect3">
<h4 id="_non_root_block_index_format_in_version_2"><a class="anchor" href="#_non_root_block_index_format_in_version_2"></a>G.2.6. Non-root block index format in version 2</h4>
<div class="paragraph">
<p>This format applies to intermediate-level and leaf index blocks of a version 2 multi-level data block index.
Every non-root index block is structured as follows.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>numEntries: the number of entries (int).</p>
</li>
<li>
<p>entryOffsets: the "secondary index" of offsets of entries in the block, to facilitate
a quick binary search on the key (<code>numEntries + 1</code> int values). The last value
is the total length of all entries in this index block. For example, in a non-root
index block with entry sizes 60, 80, 50 the "secondary index" will contain the
following int array: <code>{0, 60, 140, 190}</code>.</p>
</li>
<li>
<p>Entries.
Each entry contains:</p>
<div class="olist loweralpha">
<ol class="loweralpha" type="a">
<li>
<p>Offset of the block referenced by this entry in the file (long)</p>
</li>
<li>
<p>On-disk size of the referenced block (int)</p>
</li>
<li>
<p>Key.
The length can be calculated from entryOffsets.</p>
</li>
</ol>
</div>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_bloom_filters_in_version_2"><a class="anchor" href="#_bloom_filters_in_version_2"></a>G.2.7. Bloom filters in version 2</h4>
<div class="paragraph">
<p>In contrast with version 1, in a version 2 HFile Bloom filter metadata is stored in the load-on-open section of the HFile for quick startup.</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>A compound Bloom filter.</p>
</li>
<li>
<p>Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom filter version number 2</p>
</li>
<li>
<p>The total byte size of all compound Bloom filter chunks (long)</p>
</li>
<li>
<p>Number of hash functions (int)</p>
</li>
<li>
<p>Type of hash functions (int)</p>
</li>
<li>
<p>The total key count inserted into the Bloom filter (long)</p>
</li>
<li>
<p>The maximum total number of keys in the Bloom filter (long)</p>
</li>
<li>
<p>The number of chunks (int)</p>
</li>
<li>
<p>Comparator class used for Bloom filter keys, a UTF&gt;8 encoded string stored using Bytes.writeByteArray</p>
</li>
<li>
<p>Bloom block index in the version 2 root block index format</p>
</li>
</ol>
</div>
</div>
<div class="sect3">
<h4 id="_file_info_format_in_versions_1_and_2"><a class="anchor" href="#_file_info_format_in_versions_1_and_2"></a>G.2.8. File Info format in versions 1 and 2</h4>
<div class="paragraph">
<p>The file info block is a serialized map from byte arrays to byte arrays, with the following keys, among others.
StoreFile-level logic adds more keys to this.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.LASTKEY</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The last key of the file (byte array)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.AVG_KEY_LEN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The average key length in the file (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.AVG_VALUE_LEN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The average value length in the file (int)</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>In version 2, we did not change the file format, but we moved the file info to
the final section of the file, which can be loaded as one block when the HFile
is being opened.</p>
</div>
<div class="paragraph">
<p>Also, we do not store the comparator in the version 2 file info anymore.
Instead, we store it in the fixed file trailer.
This is because we need to know the comparator at the time of parsing the load-on-open section of the HFile.</p>
</div>
</div>
<div class="sect3">
<h4 id="_fixed_file_trailer_format_differences_between_versions_1_and_2"><a class="anchor" href="#_fixed_file_trailer_format_differences_between_versions_1_and_2"></a>G.2.9. Fixed file trailer format differences between versions 1 and 2</h4>
<div class="paragraph">
<p>The following table shows common and different fields between fixed file trailers in versions 1 and 2.
Note that the size of the trailer is different depending on the version, so it is ''fixed'' only within one version.
However, the version is always stored as the last four-byte integer in the file.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<caption class="title">Table 25. Differences between HFile Versions 1 and 2</caption>
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version 1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version 2</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">File info offset (long)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Data index offset (long)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">loadOnOpenOffset (long) /The offset of the section that we need to load when opening the file./</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Number of data index entries (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">metaIndexOffset (long) /This field is not being used by the version 1 reader, so we removed it from version 2./</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">uncompressedDataIndexSize (long) /The total uncompressed size of the whole data block index, including root-level, intermediate-level, and leaf-level blocks./</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Number of meta index entries (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Total uncompressed bytes (long)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">numEntries (int)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">numEntries (long)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The number of levels in the data block index (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">firstDataBlockOffset (long) /The offset of the first data block. Used when scanning./</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">lastDataBlockEnd (long) /The offset of the first byte after the last key/value data block. We don&#8217;t need to go beyond this offset when scanning./</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version: 1 (int)</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version: 2 (int)</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect3">
<h4 id="_getshortmidpointkey_an_optimization_for_data_index_block"><a class="anchor" href="#_getshortmidpointkey_an_optimization_for_data_index_block"></a>G.2.10. getShortMidpointKey(an optimization for data index block)</h4>
<div class="paragraph">
<p>Note: this optimization was introduced in HBase 0.95+</p>
</div>
<div class="paragraph">
<p>HFiles contain many blocks that contain a range of sorted Cells.
Each cell has a key.
To save IO when reading Cells, the HFile also has an index that maps a Cell&#8217;s start key to the offset of the beginning of a particular block.
Prior to this optimization, HBase would use the key of the first cell in each data block as the index key.</p>
</div>
<div class="paragraph">
<p>In HBASE-7845, we generate a new key that is lexicographically larger than the last key of the previous block and lexicographically equal or smaller than the start key of the current block.
While actual keys can potentially be very long, this "fake key" or "virtual key" can be much shorter.
For example, if the stop key of previous block is "the quick brown fox", the start key of current block is "the who", we could use "the r" as our virtual key in our hfile index.</p>
</div>
<div class="paragraph">
<p>There are two benefits to this:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>having shorter keys reduces the hfile index size, (allowing us to keep more indexes in memory), and</p>
</li>
<li>
<p>using something closer to the end key of the previous block allows us to avoid a potential extra IO when the target key lives in between the "virtual key" and the key of the first element in the target block.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>This optimization (implemented by the getShortMidpointKey method) is inspired by LevelDB&#8217;s ByteWiseComparatorImpl::FindShortestSeparator() and FindShortSuccessor().</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="hfilev3"><a class="anchor" href="#hfilev3"></a>G.3. HBase File Format with Security Enhancements (version 3)</h3>
<div class="paragraph">
<p>Note: this feature was introduced in HBase 0.98</p>
</div>
<div class="sect3">
<h4 id="hfilev3.motivation"><a class="anchor" href="#hfilev3.motivation"></a>G.3.1. Motivation</h4>
<div class="paragraph">
<p>Version 3 of HFile makes changes needed to ease management of encryption at rest and cell-level metadata (which in turn is needed for cell-level ACLs and cell-level visibility labels). For more information see <a href="#hbase.encryption.server">hbase.encryption.server</a>, <a href="#hbase.tags">hbase.tags</a>, <a href="#hbase.accesscontrol.configuration">hbase.accesscontrol.configuration</a>, and <a href="#hbase.visibility.labels">hbase.visibility.labels</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="hfilev3.overview"><a class="anchor" href="#hfilev3.overview"></a>G.3.2. Overview</h4>
<div class="paragraph">
<p>The version of HBase introducing the above features reads HFiles in versions 1, 2, and 3 but only writes version 3 HFiles.
Version 3 HFiles are structured the same as version 2 HFiles.
For more information see <a href="#hfilev2.overview">hfilev2.overview</a>.</p>
</div>
</div>
<div class="sect3">
<h4 id="hvilev3.infoblock"><a class="anchor" href="#hvilev3.infoblock"></a>G.3.3. File Info Block in Version 3</h4>
<div class="paragraph">
<p>Version 3 added two additional pieces of information to the reserved keys in the file info block.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.MAX_TAGS_LEN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The maximum number of bytes needed to store the serialized tags for any single cell in this hfile (int)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">hfile.TAGS_COMPRESSED</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Does the block encoder for this hfile compress tags? (boolean). Should only be present if hfile.MAX_TAGS_LEN is also present.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>When reading a Version 3 HFile the presence of <code>MAX_TAGS_LEN</code> is used to determine how to deserialize the cells within a data block.
Therefore, consumers must read the file&#8217;s info block prior to reading any data blocks.</p>
</div>
<div class="paragraph">
<p>When writing a Version 3 HFile, HBase will always include <code>MAX_TAGS_LEN</code> when flushing the memstore to underlying filesystem.</p>
</div>
<div class="paragraph">
<p>When compacting extant files, the default writer will omit <code>MAX_TAGS_LEN</code> if all of the files selected do not themselves contain any cells with tags.</p>
</div>
<div class="paragraph">
<p>See <a href="#compaction">compaction</a> for details on the compaction file selection algorithm.</p>
</div>
</div>
<div class="sect3">
<h4 id="hfilev3.datablock"><a class="anchor" href="#hfilev3.datablock"></a>G.3.4. Data Blocks in Version 3</h4>
<div class="paragraph">
<p>Within an HFile, HBase cells are stored in data blocks as a sequence of KeyValues (see <a href="#hfilev1.overview">hfilev1.overview</a>, or <a href="http://www.larsgeorge.com/2009/10/hbase-architecture-101-storage.html">Lars George&#8217;s
excellent introduction to HBase Storage</a>). In version 3, these KeyValue optionally will include a set of 0 or more tags:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version 1 &amp; 2, Version 3 without MAX_TAGS_LEN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Version 3 with MAX_TAGS_LEN</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top" colspan="2"><p class="tableblock">Key Length (4 bytes)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top" colspan="2"><p class="tableblock">Value Length (4 bytes)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top" colspan="2"><p class="tableblock">Key bytes (variable)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top" colspan="2"><p class="tableblock">Value bytes (variable)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Tags Length (2 bytes)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Tags bytes (variable)</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>If the info block for a given HFile contains an entry for <code>MAX_TAGS_LEN</code> each cell will have the length of that cell&#8217;s tags included, even if that length is zero.
The actual tags are stored as a sequence of tag length (2 bytes), tag type (1 byte), tag bytes (variable). The format an individual tag&#8217;s bytes depends on the tag type.</p>
</div>
<div class="paragraph">
<p>Note that the dependence on the contents of the info block implies that prior to reading any data blocks you must first process a file&#8217;s info block.
It also implies that prior to writing a data block you must know if the file&#8217;s info block will include <code>MAX_TAGS_LEN</code>.</p>
</div>
</div>
<div class="sect3">
<h4 id="hfilev3.fixedtrailer"><a class="anchor" href="#hfilev3.fixedtrailer"></a>G.3.5. Fixed File Trailer in Version 3</h4>
<div class="paragraph">
<p>The fixed file trailers written with HFile version 3 are always serialized with protocol buffers.
Additionally, it adds an optional field to the version 2 protocol buffer named encryption_key.
If HBase is configured to encrypt HFiles this field will store a data encryption key for this particular HFile, encrypted with the current cluster master key using AES.
For more information see <a href="#hbase.encryption.server">hbase.encryption.server</a>.</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="other.info"><a class="anchor" href="#other.info"></a>Appendix H: Other Information About HBase</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="other.info.videos"><a class="anchor" href="#other.info.videos"></a>H.1. HBase Videos</h3>
<div class="ulist">
<div class="title">Introduction to HBase</div>
<ul>
<li>
<p><a href="https://vimeo.com/23400732">Introduction to HBase</a> by Todd Lipcon (Chicago Data Summit 2011).</p>
</li>
<li>
<p><a href="https://vimeo.com/26804675">Building Real Time Services at Facebook with HBase</a> by Jonathan Gray (Berlin buzzwords 2011)</p>
</li>
<li>
<p><a href="http://www.cloudera.com/videos/hw10_video_how_stumbleupon_built_and_advertising_platform_using_hbase_and_hadoop">The Multiple Uses Of HBase</a> by Jean-Daniel Cryans(Berlin buzzwords 2011).</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="other.info.pres"><a class="anchor" href="#other.info.pres"></a>H.2. HBase Presentations (Slides)</h3>
<div class="paragraph">
<p><a href="https://www.slideshare.net/cloudera/hadoop-world-2011-advanced-hbase-schema-design-lars-george-cloudera">Advanced HBase Schema Design</a> by Lars George (Hadoop World 2011).</p>
</div>
<div class="paragraph">
<p><a href="http://www.slideshare.net/cloudera/chicago-data-summit-apache-hbase-an-introduction">Introduction to HBase</a> by Todd Lipcon (Chicago Data Summit 2011).</p>
</div>
<div class="paragraph">
<p><a href="http://www.slideshare.net/cloudera/hw09-practical-h-base-getting-the-most-from-your-h-base-install">Getting The Most From Your HBase Install</a> by Ryan Rawson, Jonathan Gray (Hadoop World 2009).</p>
</div>
</div>
<div class="sect2">
<h3 id="other.info.papers"><a class="anchor" href="#other.info.papers"></a>H.3. HBase Papers</h3>
<div class="paragraph">
<p><a href="http://research.google.com/archive/bigtable.html">BigTable</a> by Google (2006).</p>
</div>
<div class="paragraph">
<p><a href="http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html">HBase and HDFS Locality</a> by Lars George (2010).</p>
</div>
<div class="paragraph">
<p><a href="http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf">No Relation: The Mixed Blessings of Non-Relational Databases</a> by Ian Varley (2009).</p>
</div>
</div>
<div class="sect2">
<h3 id="other.info.sites"><a class="anchor" href="#other.info.sites"></a>H.4. HBase Sites</h3>
<div class="paragraph">
<p><a href="https://blog.cloudera.com/blog/category/hbase/">Cloudera&#8217;s HBase Blog</a> has a lot of links to useful HBase information.</p>
</div>
<div class="paragraph">
<p><a href="https://blog.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/">CAP Confusion</a> is a relevant entry for background information on distributed storage systems.</p>
</div>
<div class="paragraph">
<p><a href="http://refcardz.dzone.com/refcardz/hbase">HBase RefCard</a> from DZone.</p>
</div>
</div>
<div class="sect2">
<h3 id="other.info.books"><a class="anchor" href="#other.info.books"></a>H.5. HBase Books</h3>
<div class="paragraph">
<p><a href="http://shop.oreilly.com/product/0636920014348.do">HBase: The Definitive Guide</a> by Lars George.</p>
</div>
</div>
<div class="sect2">
<h3 id="other.info.books.hadoop"><a class="anchor" href="#other.info.books.hadoop"></a>H.6. Hadoop Books</h3>
<div class="paragraph">
<p><a href="http://shop.oreilly.com/product/9780596521981.do">Hadoop: The Definitive Guide</a> by Tom White.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.history"><a class="anchor" href="#hbase.history"></a>Appendix I: HBase History</h2>
<div class="sectionbody">
<div class="ulist">
<ul>
<li>
<p>2006: <a href="http://research.google.com/archive/bigtable.html">BigTable</a> paper published by Google.</p>
</li>
<li>
<p>2006 (end of year): HBase development starts.</p>
</li>
<li>
<p>2008: HBase becomes Hadoop sub-project.</p>
</li>
<li>
<p>2010: HBase becomes Apache top-level project.</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="asf"><a class="anchor" href="#asf"></a>Appendix J: HBase and the Apache Software Foundation</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase is a project in the Apache Software Foundation and as such there are responsibilities to the ASF to ensure a healthy project.</p>
</div>
<div class="sect2">
<h3 id="asf.devprocess"><a class="anchor" href="#asf.devprocess"></a>J.1. ASF Development Process</h3>
<div class="paragraph">
<p>See the <a href="https://www.apache.org/dev/#committers">Apache Development Process page</a> for all sorts of information on how the ASF is structured (e.g., PMC, committers, contributors), to tips on contributing and getting involved, and how open-source works at ASF.</p>
</div>
</div>
<div class="sect2">
<h3 id="asf.reporting"><a class="anchor" href="#asf.reporting"></a>J.2. ASF Board Reporting</h3>
<div class="paragraph">
<p>Once a quarter, each project in the ASF portfolio submits a report to the ASF board.
This is done by the HBase project lead and the committers.
See <a href="https://www.apache.org/foundation/board/reporting">ASF board reporting</a> for more information.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="orca"><a class="anchor" href="#orca"></a>Appendix K: Apache HBase Orca</h2>
<div class="sectionbody">
<div class="imageblock">
<div class="content">
<img src="images/jumping-orca_rotated_25percent.png" alt="jumping orca rotated 25percent">
</div>
<div class="title">Figure 24. Apache HBase Orca, HBase Colors, &amp; Font</div>
</div>
<div class="paragraph">
<p><a href="https://issues.apache.org/jira/browse/HBASE-4920">An Orca is the Apache HBase mascot.</a> See NOTICES.txt.
Our Orca logo we got here: <a href="http://www.vectorfree.com/jumping-orca" class="bare">http://www.vectorfree.com/jumping-orca</a> It is licensed Creative Commons Attribution 3.0.
See <a href="https://creativecommons.org/licenses/by/3.0/us/" class="bare">https://creativecommons.org/licenses/by/3.0/us/</a> We changed the logo by stripping the colored background, inverting it and then rotating it some.</p>
</div>
<div class="paragraph">
<p>The 'official' HBase color is "International Orange (Engineering)", the color of the <a href="https://en.wikipedia.org/wiki/International_orange">Golden Gate bridge</a> in San Francisco and for space suits used by NASA.</p>
</div>
<div class="paragraph">
<p>Our 'font' is <a href="http://www.dafont.com/bitsumishi.font">Bitsumishi</a>.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="tracing"><a class="anchor" href="#tracing"></a>Appendix L: Enabling Dapper-like Tracing in HBase</h2>
<div class="sectionbody">
<div class="paragraph">
<p>HBase includes facilities for tracing requests using the open source tracing library, <a href="https://htrace.incubator.apache.org/">Apache HTrace</a>.
Setting up tracing is quite simple, however it currently requires some very minor changes to your client code (this requirement may be removed in the future).</p>
</div>
<div class="paragraph">
<p>Support for this feature using HTrace 3 in HBase was added in <a href="https://issues.apache.org/jira/browse/HBASE-6449">HBASE-6449</a>. Starting with HBase 2.0, there was a non-compatible update to HTrace 4 via <a href="https://issues.apache.org/jira/browse/HBASE-18601">HBASE-18601</a>. The examples provided in this section will be using HTrace 4 package names, syntax, and conventions. For older examples, please consult previous versions of this guide.</p>
</div>
<div class="sect2">
<h3 id="tracing.spanreceivers"><a class="anchor" href="#tracing.spanreceivers"></a>L.1. SpanReceivers</h3>
<div class="paragraph">
<p>The tracing system works by collecting information in structures called 'Spans'. It is up to you to choose how you want to receive this information by implementing the <code>SpanReceiver</code> interface, which defines one method:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">public</span> <span class="type">void</span> receiveSpan(Span span);</code></pre>
</div>
</div>
<div class="paragraph">
<p>This method serves as a callback whenever a span is completed.
HTrace allows you to use as many SpanReceivers as you want so you can easily send trace information to multiple destinations.</p>
</div>
<div class="paragraph">
<p>Configure what SpanReceivers you&#8217;d like to us by putting a comma separated list of the fully-qualified class name of classes implementing <code>SpanReceiver</code> in <em>hbase-site.xml</em> property: <code>hbase.trace.spanreceiver.classes</code>.</p>
</div>
<div class="paragraph">
<p>HTrace includes a <code>LocalFileSpanReceiver</code> that writes all span information to local files in a JSON-based format.
The <code>LocalFileSpanReceiver</code> looks in <em>hbase-site.xml</em> for a <code>hbase.local-file-span-receiver.path</code> property with a value describing the name of the file to which nodes should write their span information.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">&lt;property&gt;
&lt;name&gt;hbase.trace.spanreceiver.classes&lt;/name&gt;
&lt;value&gt;org.apache.htrace.core.LocalFileSpanReceiver&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;hbase.htrace.local-file-span-receiver.path&lt;/name&gt;
&lt;value&gt;/var/log/hbase/htrace.out&lt;/value&gt;
&lt;/property&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>HTrace also provides <code>ZipkinSpanReceiver</code> which converts spans to <a href="http://github.com/twitter/zipkin">Zipkin</a> span format and send them to Zipkin server. In order to use this span receiver, you need to install the jar of htrace-zipkin to your HBase&#8217;s classpath on all of the nodes in your cluster.</p>
</div>
<div class="paragraph">
<p><em>htrace-zipkin</em> is published to the <a href="http://search.maven.org/#search%7Cgav%7C1%7Cg%3A%22org.apache.htrace%22%20AND%20a%3A%22htrace-zipkin%22">Maven central repository</a>. You could get the latest version from there or just build it locally (see the <a href="https://htrace.incubator.apache.org/">HTrace</a> homepage for information on how to do this) and then copy it out to all nodes.</p>
</div>
<div class="paragraph">
<p><code>ZipkinSpanReceiver</code> for properties called <code>hbase.htrace.zipkin.collector-hostname</code> and <code>hbase.htrace.zipkin.collector-port</code> in <em>hbase-site.xml</em> with values describing the Zipkin collector server to which span information are sent.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="xml"><span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.trace.spanreceiver.classes<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>org.apache.htrace.core.ZipkinSpanReceiver<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.htrace.zipkin.collector-hostname<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>localhost<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span>
<span class="tag">&lt;property&gt;</span>
<span class="tag">&lt;name&gt;</span>hbase.htrace.zipkin.collector-port<span class="tag">&lt;/name&gt;</span>
<span class="tag">&lt;value&gt;</span>9410<span class="tag">&lt;/value&gt;</span>
<span class="tag">&lt;/property&gt;</span></code></pre>
</div>
</div>
<div class="paragraph">
<p>If you do not want to use the included span receivers, you are encouraged to write your own receiver (take a look at <code>LocalFileSpanReceiver</code> for an example). If you think others would benefit from your receiver, file a JIRA with the HTrace project.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="tracing.client.modifications"><a class="anchor" href="#tracing.client.modifications"></a>210. Client Modifications</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In order to turn on tracing in your client code, you must initialize the module sending spans to receiver once per client process.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="directive">private</span> SpanReceiverHost spanReceiverHost;
...
Configuration conf = HBaseConfiguration.create();
SpanReceiverHost spanReceiverHost = SpanReceiverHost.getInstance(conf);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Then you simply start tracing span before requests you think are interesting, and close it when the request is done.
For example, if you wanted to trace all of your get operations, you change this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="predefined-type">Configuration</span> config = HBaseConfiguration.create();
<span class="predefined-type">Connection</span> connection = ConnectionFactory.createConnection(config);
Table table = connection.getTable(TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">t1</span><span class="delimiter">&quot;</span></span>));
Get get = <span class="keyword">new</span> Get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">r1</span><span class="delimiter">&quot;</span></span>));
<span class="predefined-type">Result</span> res = table.get(get);</code></pre>
</div>
</div>
<div class="paragraph">
<p>into:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">TraceScope ts = Trace.startSpan(<span class="string"><span class="delimiter">&quot;</span><span class="content">Gets</span><span class="delimiter">&quot;</span></span>, Sampler.ALWAYS);
<span class="keyword">try</span> {
Table table = connection.getTable(TableName.valueOf(<span class="string"><span class="delimiter">&quot;</span><span class="content">t1</span><span class="delimiter">&quot;</span></span>));
Get get = <span class="keyword">new</span> Get(Bytes.toBytes(<span class="string"><span class="delimiter">&quot;</span><span class="content">r1</span><span class="delimiter">&quot;</span></span>));
<span class="predefined-type">Result</span> res = table.get(get);
} <span class="keyword">finally</span> {
ts.close();
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>If you wanted to trace half of your 'get' operations, you would pass in:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java"><span class="keyword">new</span> ProbabilitySampler(<span class="float">0.5</span>)</code></pre>
</div>
</div>
<div class="paragraph">
<p>in lieu of <code>Sampler.ALWAYS</code> to <code>Trace.startSpan()</code>.
See the HTrace <em>README</em> for more information on Samplers.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="tracing.client.shell"><a class="anchor" href="#tracing.client.shell"></a>211. Tracing from HBase Shell</h2>
<div class="sectionbody">
<div class="paragraph">
<p>You can use <code>trace</code> command for tracing requests from HBase Shell. <code>trace 'start'</code> command turns on tracing and <code>trace 'stop'</code> command turns off tracing.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">start</span><span class="delimiter">'</span></span>
hbase(main):<span class="octal">002</span>:<span class="integer">0</span>&gt; put <span class="string"><span class="delimiter">'</span><span class="content">test</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">row1</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">f:</span><span class="delimiter">'</span></span>, <span class="string"><span class="delimiter">'</span><span class="content">val1</span><span class="delimiter">'</span></span> <span class="error">#</span> traced commands
hbase(main):<span class="octal">003</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">stop</span><span class="delimiter">'</span></span></code></pre>
</div>
</div>
<div class="paragraph">
<p><code>trace 'start'</code> and <code>trace 'stop'</code> always returns boolean value representing if or not there is ongoing tracing.
As a result, <code>trace 'stop'</code> returns false on success. <code>trace 'status'</code> just returns if or not tracing is turned on.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">hbase(main):<span class="octal">001</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">start</span><span class="delimiter">'</span></span>
=&gt; <span class="predefined-constant">true</span>
hbase(main):<span class="octal">002</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">status</span><span class="delimiter">'</span></span>
=&gt; <span class="predefined-constant">true</span>
hbase(main):<span class="octal">003</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">stop</span><span class="delimiter">'</span></span>
=&gt; <span class="predefined-constant">false</span>
hbase(main):<span class="octal">004</span>:<span class="integer">0</span>&gt; trace <span class="string"><span class="delimiter">'</span><span class="content">status</span><span class="delimiter">'</span></span>
=&gt; <span class="predefined-constant">false</span></code></pre>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="hbase.rpc"><a class="anchor" href="#hbase.rpc"></a>Appendix M: 0.95 RPC Specification</h2>
<div class="sectionbody">
<div class="paragraph">
<p>In 0.95, all client/server communication is done with <a href="https://developers.google.com/protocol-buffers/">protobuf&#8217;ed</a> Messages rather than with <a href="https://hadoop.apache.org/docs/current/api/org/apache/hadoop/io/Writable.html">Hadoop
Writables</a>.
Our RPC wire format therefore changes.
This document describes the client/server request/response protocol and our new RPC wire-format.</p>
</div>
<div class="paragraph">
<p>For what RPC is like in 0.94 and previous, see Benoît/Tsuna&#8217;s <a href="https://github.com/OpenTSDB/asynchbase/blob/master/src/HBaseRpc.java#L164">Unofficial
Hadoop / HBase RPC protocol documentation</a>.
For more background on how we arrived at this spec., see <a href="https://docs.google.com/document/d/1WCKwgaLDqBw2vpux0jPsAu2WPTRISob7HGCO8YhfDTA/edit#">HBase
RPC: WIP</a></p>
</div>
<div class="sect2">
<h3 id="_goals"><a class="anchor" href="#_goals"></a>M.1. Goals</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>A wire-format we can evolve</p>
</li>
<li>
<p>A format that does not require our rewriting server core or radically changing its current architecture (for later).</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_todo"><a class="anchor" href="#_todo"></a>M.2. TODO</h3>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>List of problems with currently specified format and where we would like to go in a version2, etc.
For example, what would we have to change if anything to move server async or to support streaming/chunking?</p>
</li>
<li>
<p>Diagram on how it works</p>
</li>
<li>
<p>A grammar that succinctly describes the wire-format.
Currently we have these words and the content of the rpc protobuf idl but a grammar for the back and forth would help with groking rpc.
Also, a little state machine on client/server interactions would help with understanding (and ensuring correct implementation).</p>
</li>
</ol>
</div>
</div>
<div class="sect2">
<h3 id="_rpc"><a class="anchor" href="#_rpc"></a>M.3. RPC</h3>
<div class="paragraph">
<p>The client will send setup information on connection establish.
Thereafter, the client invokes methods against the remote server sending a protobuf Message and receiving a protobuf Message in response.
Communication is synchronous.
All back and forth is preceded by an int that has the total length of the request/response.
Optionally, Cells(KeyValues) can be passed outside of protobufs in follow-behind Cell blocks
(because <a href="https://docs.google.com/document/d/1WEtrq-JTIUhlnlnvA0oYRLp0F8MKpEBeBSCFcQiacdw/edit#">we can&#8217;t protobuf megabytes of KeyValues</a> or Cells). These CellBlocks are encoded and optionally compressed.</p>
</div>
<div class="paragraph">
<p>For more detail on the protobufs involved, see the
<a href="https://github.com/apache/hbase/blob/master/hbase-protocol/src/main/protobuf/RPC.proto">RPC.proto</a> file in master.</p>
</div>
<div class="sect3">
<h4 id="_connection_setup"><a class="anchor" href="#_connection_setup"></a>M.3.1. Connection Setup</h4>
<div class="paragraph">
<p>Client initiates connection.</p>
</div>
<div class="sect4">
<h5 id="_client"><a class="anchor" href="#_client"></a>Client</h5>
<div class="paragraph">
<p>On connection setup, client sends a preamble followed by a connection header.</p>
</div>
<div class="listingblock">
<div class="title">&lt;preamble&gt;</div>
<div class="content">
<pre class="CodeRay highlight"><code data-lang="java">&lt;MAGIC <span class="integer">4</span> <span class="type">byte</span> integer&gt; &lt;<span class="integer">1</span> <span class="type">byte</span> RPC <span class="predefined-type">Format</span> Version&gt; &lt;<span class="integer">1</span> <span class="type">byte</span> auth type&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>We need the auth method spec.
here so the connection header is encoded if auth enabled.</p>
</div>
<div class="paragraph">
<p>E.g.: HBas0x000x50&#8201;&#8212;&#8201;4 bytes of MAGIC&#8201;&#8212;&#8201;`HBas'&#8201;&#8212;&#8201;plus one-byte of version, 0 in this case, and one byte, 0x50 (SIMPLE). of an auth type.</p>
</div>
<div class="paragraph">
<div class="title">&lt;Protobuf ConnectionHeader Message&gt;</div>
<p>Has user info, and ``protocol'', as well as the encoders and compression the client will use sending CellBlocks.
CellBlock encoders and compressors are for the life of the connection.
CellBlock encoders implement org.apache.hadoop.hbase.codec.Codec.
CellBlocks may then also be compressed.
Compressors implement org.apache.hadoop.io.compress.CompressionCodec.
This protobuf is written using writeDelimited so is prefaced by a pb varint with its serialized length</p>
</div>
</div>
<div class="sect4">
<h5 id="_server"><a class="anchor" href="#_server"></a>Server</h5>
<div class="paragraph">
<p>After client sends preamble and connection header, server does NOT respond if successful connection setup.
No response means server is READY to accept requests and to give out response.
If the version or authentication in the preamble is not agreeable or the server has trouble parsing the preamble, it will throw a org.apache.hadoop.hbase.ipc.FatalConnectionException explaining the error and will then disconnect.
If the client in the connection header&#8201;&#8212;&#8201;i.e.
the protobuf&#8217;d Message that comes after the connection preamble&#8201;&#8212;&#8201;asks for a Service the server does not support or a codec the server does not have, again we throw a FatalConnectionException with explanation.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_request"><a class="anchor" href="#_request"></a>M.3.2. Request</h4>
<div class="paragraph">
<p>After a Connection has been set up, client makes requests.
Server responds.</p>
</div>
<div class="paragraph">
<p>A request is made up of a protobuf RequestHeader followed by a protobuf Message parameter.
The header includes the method name and optionally, metadata on the optional CellBlock that may be following.
The parameter type suits the method being invoked: i.e.
if we are doing a getRegionInfo request, the protobuf Message param will be an instance of GetRegionInfoRequest.
The response will be a GetRegionInfoResponse.
The CellBlock is optionally used ferrying the bulk of the RPC data: i.e. Cells/KeyValues.</p>
</div>
<div class="sect4">
<h5 id="_request_parts"><a class="anchor" href="#_request_parts"></a>Request Parts</h5>
<div class="paragraph">
<div class="title">&lt;Total Length&gt;</div>
<p>The request is prefaced by an int that holds the total length of what follows.</p>
</div>
<div class="paragraph">
<div class="title">&lt;Protobuf RequestHeader Message&gt;</div>
<p>Will have call.id, trace.id, and method name, etc.
including optional Metadata on the Cell block IFF one is following.
Data is protobuf&#8217;d inline in this pb Message or optionally comes in the following CellBlock</p>
</div>
<div class="paragraph">
<div class="title">&lt;Protobuf Param Message&gt;</div>
<p>If the method being invoked is getRegionInfo, if you study the Service descriptor for the client to regionserver protocol, you will find that the request sends a GetRegionInfoRequest protobuf Message param in this position.</p>
</div>
<div class="paragraph">
<div class="title">&lt;CellBlock&gt;</div>
<p>An encoded and optionally compressed Cell block.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_response"><a class="anchor" href="#_response"></a>M.3.3. Response</h4>
<div class="paragraph">
<p>Same as Request, it is a protobuf ResponseHeader followed by a protobuf Message response where the Message response type suits the method invoked.
Bulk of the data may come in a following CellBlock.</p>
</div>
<div class="sect4">
<h5 id="_response_parts"><a class="anchor" href="#_response_parts"></a>Response Parts</h5>
<div class="paragraph">
<div class="title">&lt;Total Length&gt;</div>
<p>The response is prefaced by an int that holds the total length of what follows.</p>
</div>
<div class="paragraph">
<div class="title">&lt;Protobuf ResponseHeader Message&gt;</div>
<p>Will have call.id, etc.
Will include exception if failed processing.
Optionally includes metadata on optional, IFF there is a CellBlock following.</p>
</div>
<div class="paragraph">
<div class="title">&lt;Protobuf Response Message&gt;</div>
<p>Return or may be nothing if exception.
If the method being invoked is getRegionInfo, if you study the Service descriptor for the client to regionserver protocol, you will find that the response sends a GetRegionInfoResponse protobuf Message param in this position.</p>
</div>
<div class="paragraph">
<div class="title">&lt;CellBlock&gt;</div>
<p>An encoded and optionally compressed Cell block.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_exceptions"><a class="anchor" href="#_exceptions"></a>M.3.4. Exceptions</h4>
<div class="paragraph">
<p>There are two distinct types.
There is the request failed which is encapsulated inside the response header for the response.
The connection stays open to receive new requests.
The second type, the FatalConnectionException, kills the connection.</p>
</div>
<div class="paragraph">
<p>Exceptions can carry extra information.
See the ExceptionResponse protobuf type.
It has a flag to indicate do-no-retry as well as other miscellaneous payload to help improve client responsiveness.</p>
</div>
</div>
<div class="sect3">
<h4 id="_cellblocks"><a class="anchor" href="#_cellblocks"></a>M.3.5. CellBlocks</h4>
<div class="paragraph">
<p>These are not versioned.
Server can do the codec or it cannot.
If new version of a codec with say, tighter encoding, then give it a new class name.
Codecs will live on the server for all time so old clients can connect.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="_notes_2"><a class="anchor" href="#_notes_2"></a>M.4. Notes</h3>
<div class="paragraph">
<div class="title">Constraints</div>
<p>In some part, current wire-format&#8201;&#8212;&#8201;i.e.
all requests and responses preceded by a length&#8201;&#8212;&#8201;has been dictated by current server non-async architecture.</p>
</div>
<div class="paragraph">
<div class="title">One fat pb request or header+param</div>
<p>We went with pb header followed by pb param making a request and a pb header followed by pb response for now.
Doing header+param rather than a single protobuf Message with both header and param content:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Is closer to what we currently have</p>
</li>
<li>
<p>Having a single fat pb requires extra copying putting the already pb&#8217;d param into the body of the fat request pb (and same making result)</p>
</li>
<li>
<p>We can decide whether to accept the request or not before we read the param; for example, the request might be low priority.
As is, we read header+param in one go as server is currently implemented so this is a TODO.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>The advantages are minor.
If later, fat request has clear advantage, can roll out a v2 later.</p>
</div>
<div class="sect3">
<h4 id="rpc.configs"><a class="anchor" href="#rpc.configs"></a>M.4.1. RPC Configurations</h4>
<div class="paragraph">
<div class="title">CellBlock Codecs</div>
<p>To enable a codec other than the default <code>KeyValueCodec</code>, set <code>hbase.client.rpc.codec</code> to the name of the Codec class to use.
Codec must implement hbase&#8217;s <code>Codec</code> Interface.
After connection setup, all passed cellblocks will be sent with this codec.
The server will return cellblocks using this same codec as long as the codec is on the servers' CLASSPATH (else you will get <code>UnsupportedCellCodecException</code>).</p>
</div>
<div class="paragraph">
<p>To change the default codec, set <code>hbase.client.default.rpc.codec</code>.</p>
</div>
<div class="paragraph">
<p>To disable cellblocks completely and to go pure protobuf, set the default to the empty String and do not specify a codec in your Configuration.
So, set <code>hbase.client.default.rpc.codec</code> to the empty string and do not set <code>hbase.client.rpc.codec</code>.
This will cause the client to connect to the server with no codec specified.
If a server sees no codec, it will return all responses in pure protobuf.
Running pure protobuf all the time will be slower than running with cellblocks.</p>
</div>
<div class="paragraph">
<div class="title">Compression</div>
<p>Uses hadoop&#8217;s compression codecs.
To enable compressing of passed CellBlocks, set <code>hbase.client.rpc.compressor</code> to the name of the Compressor to use.
Compressor must implement Hadoop&#8217;s CompressionCodec Interface.
After connection setup, all passed cellblocks will be sent compressed.
The server will return cellblocks compressed using this same compressor as long as the compressor is on its CLASSPATH (else you will get <code>UnsupportedCompressionCodecException</code>).</p>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="_known_incompatibilities_among_hbase_versions"><a class="anchor" href="#_known_incompatibilities_among_hbase_versions"></a>Appendix N: Known Incompatibilities Among HBase Versions</h2>
<div class="sectionbody">
</div>
</div>
<div class="sect1">
<h2 id="_hbase_2_0_incompatible_changes"><a class="anchor" href="#_hbase_2_0_incompatible_changes"></a>212. HBase 2.0 Incompatible Changes</h2>
<div class="sectionbody">
<div class="paragraph">
<p>This appendix describes incompatible changes from earlier versions of HBase against HBase 2.0.
This list is not meant to be wholly encompassing of all possible incompatibilities.
Instead, this content is intended to give insight into some obvious incompatibilities which most
users will face coming from HBase 1.x releases.</p>
</div>
<div class="sect2">
<h3 id="_list_of_major_changes_for_hbase_2_0"><a class="anchor" href="#_list_of_major_changes_for_hbase_2_0"></a>212.1. List of Major Changes for HBase 2.0</h3>
<div class="ulist">
<ul>
<li>
<p>HBASE-1912- HBCK is a HBase database checking tool for capturing the inconsistency. As an HBase administrator, you should not use HBase version 1.0 hbck tool to check the HBase 2.0 database. Doing so will break the database and throw an exception error.</p>
</li>
<li>
<p>HBASE-16189 and HBASE-18945- You cannot open the HBase 2.0 hfiles through HBase 1.0 version. If you are an admin or an HBase user who is using HBase version 1.x, you must first do a rolling upgrade to the latest version of HBase 1.x and then upgrade to HBase 2.0.</p>
</li>
<li>
<p>HBASE-18240 - Changed the ReplicationEndpoint Interface. It also introduces a new hbase-third party 1.0 that packages all the third party utilities, which are expected to run in the hbase cluster.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="_coprocessor_api_changes"><a class="anchor" href="#_coprocessor_api_changes"></a>212.2. Coprocessor API changes</h3>
<div class="ulist">
<ul>
<li>
<p>HBASE-16769 - Deprecated PB references from MasterObserver and RegionServerObserver.</p>
</li>
<li>
<p>HBASE-17312 - [JDK8] Use default method for Observer Coprocessors. The interface classes of BaseMasterAndRegionObserver, BaseMasterObserver, BaseRegionObserver, BaseRegionServerObserver and BaseWALObserver uses JDK8&#8217;s 'default' keyword to provide empty and no-op implementations.</p>
</li>
<li>
<p>Interface HTableInterface
HBase 2.0 introduces following changes to the methods listed below:</p>
</li>
</ul>
</div>
<div class="sect3">
<h4 id="__interface_coprocessorenvironment_changes_2"><a class="anchor" href="#__interface_coprocessorenvironment_changes_2"></a>212.2.1. [−] interface CoprocessorEnvironment changes (2)</h4>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getTable ( TableName ) has been removed.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getTable ( TableName, ExecutorService ) has been removed.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>Public Audience</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>The following tables describes the coprocessor changes.</p>
</div>
<div class="sect4">
<h5 id="__class_coprocessorrpcchannel_1"><a class="anchor" href="#__class_coprocessorrpcchannel_1"></a>[−] class CoprocessorRpcChannel (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class has become interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception depending on the usage of this class.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="_class_coprocessorhost_e"><a class="anchor" href="#_class_coprocessorhost_e"></a>Class CoprocessorHost&lt;E&gt;</h5>
<div class="paragraph">
<p>Classes that were Audience Private but were removed.</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Type of field coprocessors has been changed from java.util.SortedSet&lt;E&gt; to org.apache.hadoop.hbase.util.SortedList&lt;E&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_masterobserver"><a class="anchor" href="#_masterobserver"></a>212.2.2. MasterObserver</h4>
<div class="paragraph">
<p>HBase 2.0 introduces following changes to the MasterObserver interface.</p>
</div>
<div class="sect4">
<h5 id="__interface_masterobserver_14"><a class="anchor" href="#__interface_masterobserver_14"></a>[−] interface MasterObserver (14)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostCloneSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostCreateTable ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostDeleteSnapshot (ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostGetTableDescriptors ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, List&lt;HTableDescriptor&gt; ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostModifyTable ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, TableName, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostRestoreSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreCloneSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreCreateTable ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreDeleteSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreGetTableDescriptors ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, List&lt;TableName&gt;, List&lt;HTableDescriptor&gt; ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreModifyTable ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, TableName, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreRestoreSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreSnapshot ( ObserverContext&lt;MasterCoprocessorEnvironment&gt;, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodErrorexception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_regionobserver"><a class="anchor" href="#_regionobserver"></a>212.2.3. RegionObserver</h4>
<div class="paragraph">
<p>HBase 2.0 introduces following changes to the RegionObserver interface.</p>
</div>
<div class="sect4">
<h5 id="__interface_regionobserver_13"><a class="anchor" href="#__interface_regionobserver_13"></a>[−] interface RegionObserver (13)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostCloseRegionOperation ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, HRegion.Operation ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostCompactSelection ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, Store, ImmutableList&lt;StoreFile&gt; ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostCompactSelection ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, Store, ImmutableList&lt;StoreFile&gt;, CompactionRequest ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostGetClosestRowBefore ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, byte[ ], byte[ ], Result ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method DeleteTrackerpostInstantiateDeleteTracker ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, DeleteTracker ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostSplit ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, HRegion, HRegion ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostStartRegionOperation ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, HRegion.Operation ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method StoreFile.ReaderpostStoreFileReaderOpen ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostWALRestore ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method InternalScannerpreFlushScannerOpen ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, Store, KeyValueScanner, InternalScanner ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreGetClosestRowBefore ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, byte[ ], byte[ ], Result ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method StoreFile.ReaderpreStoreFileReaderOpen ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpreWALRestore ( ObserverContext&lt;RegionCoprocessorEnvironment&gt;, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_walobserver"><a class="anchor" href="#_walobserver"></a>212.2.4. WALObserver</h4>
<div class="paragraph">
<p>HBase 2.0 introduces following changes to the WALObserver interface.</p>
</div>
<div class="sect4">
<h5 id="__interface_walobserver"><a class="anchor" href="#__interface_walobserver"></a>[−] interface WALObserver</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method voidpostWALWrite ( ObserverContext&lt;WALCoprocessorEnvironment&gt;, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method booleanpreWALWrite ( ObserverContext&lt;WALCoprocessorEnvironment&gt;, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_miscellaneous"><a class="anchor" href="#_miscellaneous"></a>212.2.5. Miscellaneous</h4>
<div class="paragraph">
<p>HBase 2.0 introduces changes to the following classes:</p>
</div>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, OnlineRegions.class package org.apache.hadoop.hbase.regionserver</p>
</div>
<div class="sect4">
<h5 id="__onlineregions_getfromonlineregions_string_p1_abstract_hregion"><a class="anchor" href="#__onlineregions_getfromonlineregions_string_p1_abstract_hregion"></a>[−] OnlineRegions.getFromOnlineRegions ( String p1 ) [abstract] : HRegion</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/OnlineRegions.getFromOnlineRegions:(Ljava/lang/String;)Lorg/apache/hadoop/hbase/regionserver/HRegion;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from Region to Region.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, RegionCoprocessorEnvironment.class package org.apache.hadoop.hbase.coprocessor</p>
</div>
</div>
<div class="sect4">
<h5 id="__regioncoprocessorenvironment_getregion_abstract_hregion"><a class="anchor" href="#__regioncoprocessorenvironment_getregion_abstract_hregion"></a>[−] RegionCoprocessorEnvironment.getRegion ( ) [abstract] : HRegion</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.getRegion:()Lorg/apache/hadoop/hbase/regionserver/HRegion;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.regionserver.HRegion to org.apache.hadoop.hbase.regionserver.Region.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, RegionCoprocessorHost.class package org.apache.hadoop.hbase.regionserver</p>
</div>
</div>
<div class="sect4">
<h5 id="__regioncoprocessorhost_postappend_append_append_result_result_void"><a class="anchor" href="#__regioncoprocessorhost_postappend_append_append_result_result_void"></a>[−] RegionCoprocessorHost.postAppend ( Append append, Result result ) : void</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.postAppend:(Lorg/apache/hadoop/hbase/client/Append;Lorg/apache/hadoop/hbase/client/Result;)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to org.apache.hadoop.hbase.client.Result.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__regioncoprocessorhost_prestorefilereaderopen_filesystem_fs_path_p_fsdatainputstreamwrapper_in_long_size_cacheconfig_cacheconf_reference_r_storefile_reader"><a class="anchor" href="#__regioncoprocessorhost_prestorefilereaderopen_filesystem_fs_path_p_fsdatainputstreamwrapper_in_long_size_cacheconfig_cacheconf_reference_r_storefile_reader"></a>[−] RegionCoprocessorHost.preStoreFileReaderOpen ( FileSystem fs, Path p, FSDataInputStreamWrapper in, long size,CacheConfig cacheConf, Reference r ) : StoreFile.Reader</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.preStoreFileReaderOpen:(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;Lorg/apache/hadoop/hbase/io/FSDataInputStreamWrapper;JLorg/apache/hadoop/hbase/io/hfile/CacheConfig;Lorg/apache/hadoop/hbase/io/Reference;)Lorg/apache/hadoop/hbase/regionserver/StoreFile$Reader;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from StoreFile.Reader to StoreFileReader.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_ipc"><a class="anchor" href="#_ipc"></a>212.2.6. IPC</h4>
</div>
<div class="sect3">
<h4 id="_scheduler_changes"><a class="anchor" href="#_scheduler_changes"></a>212.2.7. Scheduler changes:</h4>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Following methods became abstract:</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>package org.apache.hadoop.hbase.ipc</p>
</div>
<div class="sect4">
<h5 id="__class_rpcscheduler_1"><a class="anchor" href="#__class_rpcscheduler_1"></a>[−]class RpcScheduler (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method void dispatch ( CallRunner ) has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, RpcScheduler.class package org.apache.hadoop.hbase.ipc</p>
</div>
</div>
<div class="sect4">
<h5 id="__rpcscheduler_dispatch_callrunner_p1_abstract_void_1"><a class="anchor" href="#__rpcscheduler_dispatch_callrunner_p1_abstract_void_1"></a>[−] RpcScheduler.dispatch ( CallRunner p1 ) [abstract] : void 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ipc/RpcScheduler.dispatch:(Lorg/apache/hadoop/hbase/ipc/CallRunner;)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to boolean.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Following abstract methods have been removed:</p>
</li>
</ol>
</div>
</div>
<div class="sect4">
<h5 id="__interface_priorityfunction_2"><a class="anchor" href="#__interface_priorityfunction_2"></a>[−]interface PriorityFunction (2)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method longgetDeadline ( RPCProtos.RequestHeader, Message ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method int getPriority ( RPCProtos.RequestHeader, Message ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_server_api_changes"><a class="anchor" href="#_server_api_changes"></a>212.2.8. Server API changes:</h4>
<div class="sect4">
<h5 id="__class_rpcserver_12"><a class="anchor" href="#__class_rpcserver_12"></a>[−] class RpcServer (12)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Type of field CurCall has been changed from java.lang.ThreadLocal&lt;RpcServer.Call&gt; to java.lang.ThreadLocal&lt;RpcCall&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class became abstract.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by InstantiationError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method int getNumOpenConnections ( ) has been added to this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class became abstract and a client program may be interrupted by InstantiationError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field callQueueSize of type org.apache.hadoop.hbase.util.Counter has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field connectionList of type java.util.List&lt;RpcServer.Connection&gt; has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field maxIdleTime of type int has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field numConnections of type int has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field port of type int has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field purgeTimeout of type long has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field responder of type RpcServer.Responder has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field socketSendBufferSize of type int has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field thresholdIdleConnections of type int has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following abstract method has been removed:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method Pair&lt;Message,CellScanner&gt;call ( BlockingService, Descriptors.MethodDescriptor, Message, CellScanner, long, MonitoredRPCHandler ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_replication_and_wal_changes"><a class="anchor" href="#_replication_and_wal_changes"></a>212.2.9. Replication and WAL changes:</h4>
<div class="paragraph">
<p>HBASE-18733: WALKey has been purged completely in HBase 2.0.
Following are the changes to the WALKey:</p>
</div>
<div class="sect4">
<h5 id="__classwalkey_8"><a class="anchor" href="#__classwalkey_8"></a>[−] classWALKey (8)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access level of field clusterIds has been changed from protected to private.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IllegalAccessError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access level of field compressionContext has been changed from protected to private.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IllegalAccessError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access level of field encodedRegionName has been changed from protected to private.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IllegalAccessError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access level of field tablename has been changed from protectedto private.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IllegalAccessError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Access level of field writeTime has been changed from protectedto private.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IllegalAccessError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following fields have been removed:</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field LOG of type org.apache.commons.logging.Log has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field VERSION of type WALKey.Version has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field logSeqNum of type long has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following are the changes to the WALEdit.class:
hbase-server-1.0.0.jar, WALEdit.class package org.apache.hadoop.hbase.regionserver.wal</p>
</div>
</div>
<div class="sect4">
<h5 id="_waledit_getcompaction_cell_kv_static_walprotos_compactiondescriptor_1"><a class="anchor" href="#_waledit_getcompaction_cell_kv_static_walprotos_compactiondescriptor_1"></a>WALEdit.getCompaction ( Cell kv ) [static] : WALProtos.CompactionDescriptor (1)</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/wal/WALEdit.getCompaction:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$CompactionDescriptor;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="_waledit_getflushdescriptor_cell_cell_static_walprotos_flushdescriptor_1"><a class="anchor" href="#_waledit_getflushdescriptor_cell_cell_static_walprotos_flushdescriptor_1"></a>WALEdit.getFlushDescriptor ( Cell cell ) [static] : WALProtos.FlushDescriptor (1)</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/wal/WALEdit.getFlushDescriptor:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$FlushDescriptor;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="_waledit_getregioneventdescriptor_cell_cell_static_walprotos_regioneventdescriptor_1"><a class="anchor" href="#_waledit_getregioneventdescriptor_cell_cell_static_walprotos_regioneventdescriptor_1"></a>WALEdit.getRegionEventDescriptor ( Cell cell ) [static] : WALProtos.RegionEventDescriptor (1)</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/regionserver/wal/WALEdit.getRegionEventDescriptor:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$RegionEventDescriptor;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following is the change to the WALKey.class:
package org.apache.hadoop.hbase.wal</p>
</div>
</div>
<div class="sect4">
<h5 id="_walkey_getbuilder_walcellcodec_bytestringcompressor_compressor_walprotos_walkey_builder_1"><a class="anchor" href="#_walkey_getbuilder_walcellcodec_bytestringcompressor_compressor_walprotos_walkey_builder_1"></a>WALKey.getBuilder ( WALCellCodec.ByteStringCompressor compressor ) : WALProtos.WALKey.Builder 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/wal/WALKey.getBuilder:(Lorg/apache/hadoop/hbase/regionserver/wal/WALCellCodec$ByteStringCompressor;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$WALKey$Builder;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey.Builder.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_deprecated_apis_or_coprocessor"><a class="anchor" href="#_deprecated_apis_or_coprocessor"></a>212.2.10. Deprecated APIs or coprocessor:</h4>
<div class="paragraph">
<p>HBASE-16769 - PB references from MasterObserver and RegionServerObserver has been removed.</p>
</div>
</div>
<div class="sect3">
<h4 id="_admin_interface_api_changes"><a class="anchor" href="#_admin_interface_api_changes"></a>212.2.11. Admin Interface API changes:</h4>
<div class="paragraph">
<p>You cannot administer an HBase 2.0 cluster with an HBase 1.0 client that includes RelicationAdmin, ACC, Thrift and REST usage of Admin ops. Methods returning protobufs have been changed to return POJOs instead. pb is not used in the APIs anymore. Returns have changed from void to Future for async methods.
HBASE-18106 - Admin.listProcedures and Admin.listLocks were renamed to getProcedures and getLocks.
MapReduce makes use of Admin doing following admin.getClusterStatus() to calcluate Splits.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Thrift usage of Admin API:
compact(ByteBuffer)
createTable(ByteBuffer, List&lt;ColumnDescriptor&gt;)
deleteTable(ByteBuffer)
disableTable(ByteBuffer)
enableTable(ByteBuffer)
getTableNames()
majorCompact(ByteBuffer)</p>
</li>
<li>
<p>REST usage of Admin API:
hbase-rest
org.apache.hadoop.hbase.rest
RootResource
getTableList()
TableName[] tableNames = servlet.getAdmin().listTableNames();
SchemaResource
delete(UriInfo)
Admin admin = servlet.getAdmin();
update(TableSchemaModel, boolean, UriInfo)
Admin admin = servlet.getAdmin();
StorageClusterStatusResource
get(UriInfo)
ClusterStatus status = servlet.getAdmin().getClusterStatus();
StorageClusterVersionResource
get(UriInfo)
model.setVersion(servlet.getAdmin().getClusterStatus().getHBaseVersion());
TableResource
exists()
return servlet.getAdmin().tableExists(TableName.valueOf(table));</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Following are the changes to the Admin interface:</p>
</div>
<div class="sect4">
<h5 id="__interface_admin_9"><a class="anchor" href="#__interface_admin_9"></a>[−] interface Admin (9)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method createTableAsync ( HTableDescriptor, byte[ ][ ] ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method disableTableAsync ( TableName ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method enableTableAsync ( TableName ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getCompactionState ( TableName ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getCompactionStateForRegion ( byte[ ] ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method isSnapshotFinished ( HBaseProtos.SnapshotDescription ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method snapshot ( String, TableName, HBaseProtos.SnapshotDescription.Type ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method snapshot ( HBaseProtos.SnapshotDescription ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method takeSnapshotAsync ( HBaseProtos.SnapshotDescription ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Following are the changes to the Admin.class:
hbase-client-1.0.0.jar, Admin.class package org.apache.hadoop.hbase.client</p>
</div>
</div>
<div class="sect4">
<h5 id="__admin_createtableasync_htabledescriptor_p1_byte_p2_abstract_void_1"><a class="anchor" href="#__admin_createtableasync_htabledescriptor_p1_byte_p2_abstract_void_1"></a>[−] Admin.createTableAsync ( HTableDescriptor p1, byte[ ][ ] p2 ) [abstract] : void 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Admin.createTableAsync:(Lorg/apache/hadoop/hbase/HTableDescriptor;[[B)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to java.util.concurrent.Future&lt;java.lang.Void&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__admin_disabletableasync_tablename_p1_abstract_void_1"><a class="anchor" href="#__admin_disabletableasync_tablename_p1_abstract_void_1"></a>[−] Admin.disableTableAsync ( TableName p1 ) [abstract] : void 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Admin.disableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to java.util.concurrent.Future&lt;java.lang.Void&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="_admin_enabletableasync_tablename_p1_abstract_void_1"><a class="anchor" href="#_admin_enabletableasync_tablename_p1_abstract_void_1"></a>Admin.enableTableAsync ( TableName p1 ) [abstract] : void 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Admin.enableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to java.util.concurrent.Future&lt;java.lang.Void&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__admin_getcompactionstate_tablename_p1_abstract_adminprotos_getregioninforesponse_compactionstate_1"><a class="anchor" href="#__admin_getcompactionstate_tablename_p1_abstract_adminprotos_getregioninforesponse_compactionstate_1"></a>[−] Admin.getCompactionState ( TableName p1 ) [abstract] : AdminProtos.GetRegionInfoResponse.CompactionState 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Admin.getCompactionState:(Lorg/apache/hadoop/hbase/TableName;)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__admin_getcompactionstateforregion_byte_p1_abstract_adminprotos_getregioninforesponse_compactionstate_1"><a class="anchor" href="#__admin_getcompactionstateforregion_byte_p1_abstract_adminprotos_getregioninforesponse_compactionstate_1"></a>[−] Admin.getCompactionStateForRegion ( byte[ ] p1 ) [abstract] : AdminProtos.GetRegionInfoResponse.CompactionState 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Admin.getCompactionStateForRegion:([B)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_htabledescriptor_and_hcolumndescriptor_changes"><a class="anchor" href="#_htabledescriptor_and_hcolumndescriptor_changes"></a>212.2.12. HTableDescriptor and HColumnDescriptor changes</h4>
<div class="paragraph">
<p>HTableDescriptor and HColumnDescriptor has become interfaces and you can create it through Builders. HCD has become CFD. It no longer implements writable interface.
package org.apache.hadoop.hbase</p>
</div>
<div class="sect4">
<h5 id="__class_hcolumndescriptor_1"><a class="anchor" href="#__class_hcolumndescriptor_1"></a>[−] class HColumnDescriptor (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Removed super-interface org.apache.hadoop.io.WritableComparable&lt;HColumnDescriptor&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>HColumnDescriptor in 1.0.0
{code}
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HColumnDescriptor implements WritableComparable&lt;HColumnDescriptor&gt; {
{code}</p>
</div>
<div class="paragraph">
<p>HColumnDescriptor in 2.0
{code}
@InterfaceAudience.Public
@Deprecated // remove it in 3.0
public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable&lt;HColumnDescriptor&gt; {
{code}</p>
</div>
<div class="paragraph">
<p>For META_TABLEDESC, the maker method had been deprecated already in HTD in 1.0.0. OWNER_KEY is still in HTD.</p>
</div>
</div>
<div class="sect4">
<h5 id="_class_htabledescriptor_3"><a class="anchor" href="#_class_htabledescriptor_3"></a>class HTableDescriptor (3)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Removed super-interface org.apache.hadoop.io.WritableComparable&lt;HTableDescriptor&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field META_TABLEDESC of type HTableDescriptor has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>hbase-client-1.0.0.jar, HTableDescriptor.class package org.apache.hadoop.hbase</p>
</div>
</div>
<div class="sect4">
<h5 id="__htabledescriptor_getcolumnfamilies_hcolumndescriptor_1"><a class="anchor" href="#__htabledescriptor_getcolumnfamilies_hcolumndescriptor_1"></a>[−] HTableDescriptor.getColumnFamilies ( ) : HColumnDescriptor[ ] (1)</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/HTableDescriptor.getColumnFamilies:()[Lorg/apache/hadoop/hbase/HColumnDescriptor;</p>
</div>
</div>
<div class="sect4">
<h5 id="__class_hcolumndescriptor_1_2"><a class="anchor" href="#__class_hcolumndescriptor_1_2"></a>[−] class HColumnDescriptor (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from HColumnDescriptor[]to client.ColumnFamilyDescriptor[].</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__htabledescriptor_getcoprocessors_list_string_1"><a class="anchor" href="#__htabledescriptor_getcoprocessors_list_string_1"></a>[−] HTableDescriptor.getCoprocessors ( ) : List&lt;String&gt; (1)</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/HTableDescriptor.getCoprocessors:()Ljava/util/List;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from java.util.List&lt;java.lang.String&gt; to java.util.Collection.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-12990 MetaScanner is removed and it is replaced by MetaTableAccessor.</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="_htablewrapper_changes"><a class="anchor" href="#_htablewrapper_changes"></a>HTableWrapper changes:</h5>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, HTableWrapper.class package org.apache.hadoop.hbase.client</p>
</div>
</div>
<div class="sect4">
<h5 id="__htablewrapper_createwrapper_list_htableinterface_opentables_tablename_tablename_coprocessorhost_environment_env_executorservice_pool_static_htableinterface_1"><a class="anchor" href="#__htablewrapper_createwrapper_list_htableinterface_opentables_tablename_tablename_coprocessorhost_environment_env_executorservice_pool_static_htableinterface_1"></a>[−] HTableWrapper.createWrapper ( List&lt;HTableInterface&gt; openTables, TableName tableName, CoprocessorHost.Environment env, ExecutorService pool ) [static] : HTableInterface 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/HTableWrapper.createWrapper:(Ljava/util/List;Lorg/apache/hadoop/hbase/TableName;Lorg/apache/hadoop/hbase/coprocessor/CoprocessorHost$Environment;Ljava/util/concurrent/ExecutorService;)Lorg/apache/hadoop/hbase/client/HTableInterface;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from HTableInterface to Table.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-12586: Delete all public HTable constructors and delete ConnectionManager#{delete,get}Connection.</p>
</li>
<li>
<p>HBASE-9117: Remove HTablePool and all HConnection pooling related APIs.</p>
</li>
<li>
<p>HBASE-13214: Remove deprecated and unused methods from HTable class
Following are the changes to the Table interface:</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__interface_table_4"><a class="anchor" href="#__interface_table_4"></a>[−] interface Table (4)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method batch ( List&lt;?&gt; ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method batchCallback ( List&lt;?&gt;, Batch.Callback&lt;R&gt; )has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getWriteBufferSize ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method setWriteBufferSize ( long ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_deprecated_buffer_methods_in_table_in_1_0_1_and_removed_in_2_0_0"><a class="anchor" href="#_deprecated_buffer_methods_in_table_in_1_0_1_and_removed_in_2_0_0"></a>212.2.13. Deprecated buffer methods in Table (in 1.0.1) and removed in 2.0.0</h4>
<div class="ulist">
<ul>
<li>
<p>HBASE-13298- Clarify if Table.{set|get}WriteBufferSize() is deprecated or not.</p>
</li>
<li>
<p>LockTimeoutException and OperationConflictException classes have been removed.</p>
</li>
</ul>
</div>
</div>
<div class="sect3">
<h4 id="_class_operationconflictexception_1"><a class="anchor" href="#_class_operationconflictexception_1"></a>212.2.14. class OperationConflictException (1)</h4>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class has been removed.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoClassDefFoundErrorexception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect3">
<h4 id="_class_class_locktimeoutexception_1"><a class="anchor" href="#_class_class_locktimeoutexception_1"></a>212.2.15. class class LockTimeoutException (1)</h4>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class has been removed.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoClassDefFoundErrorexception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect3">
<h4 id="_filter_api_changes"><a class="anchor" href="#_filter_api_changes"></a>212.2.16. Filter API changes:</h4>
<div class="paragraph">
<p>Following methods have been removed:
package org.apache.hadoop.hbase.filter</p>
</div>
<div class="sect4">
<h5 id="__class_filter_2"><a class="anchor" href="#__class_filter_2"></a>[−] class Filter (2)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getNextKeyHint ( KeyValue ) has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method transform ( KeyValue ) has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-12296 Filters should work with ByteBufferedCell.</p>
</li>
<li>
<p>HConnection is removed in HBase 2.0.</p>
</li>
<li>
<p>RegionLoad and ServerLoad internally moved to shaded PB.</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__class_regionload_1"><a class="anchor" href="#__class_regionload_1"></a>[−] class RegionLoad (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Type of field regionLoadPB has been changed from protobuf.generated.ClusterStatusProtos.RegionLoad to shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-15783:AccessControlConstants#OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST is not used any more.
package org.apache.hadoop.hbase.security.access</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__interface_accesscontrolconstants_3"><a class="anchor" href="#__interface_accesscontrolconstants_3"></a>[−] interface AccessControlConstants (3)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field OP_ATTRIBUTE_ACL_STRATEGY of type java.lang.Stringhas been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST of type byte[] has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field OP_ATTRIBUTE_ACL_STRATEGY_DEFAULT of type byte[] has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="_serverload_returns_long_instead_of_int_1"><a class="anchor" href="#_serverload_returns_long_instead_of_int_1"></a>ServerLoad returns long instead of int 1</h5>
<div class="paragraph">
<p>hbase-client-1.0.0.jar, ServerLoad.class package org.apache.hadoop.hbase</p>
</div>
</div>
<div class="sect4">
<h5 id="__serverload_getnumberofrequests_int_1"><a class="anchor" href="#__serverload_getnumberofrequests_int_1"></a>[−] ServerLoad.getNumberOfRequests ( ) : int 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ServerLoad.getNumberOfRequests:()I</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from int to long.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__serverload_getreadrequestscount_int_1"><a class="anchor" href="#__serverload_getreadrequestscount_int_1"></a>[−] ServerLoad.getReadRequestsCount ( ) : int 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ServerLoad.getReadRequestsCount:()I</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from int to long.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__serverload_gettotalnumberofrequests_int_1"><a class="anchor" href="#__serverload_gettotalnumberofrequests_int_1"></a>[−] ServerLoad.getTotalNumberOfRequests ( ) : int 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ServerLoad.getTotalNumberOfRequests:()I</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from int to long.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
<div class="sect4">
<h5 id="__serverload_getwriterequestscount_int_1"><a class="anchor" href="#__serverload_getwriterequestscount_int_1"></a>[−]ServerLoad.getWriteRequestsCount ( ) : int 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ServerLoad.getWriteRequestsCount:()I</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from int to long.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-13636 Remove deprecation for HBASE-4072 (Reading of zoo.cfg)</p>
</li>
<li>
<p>HConstants are removed. HBASE-16040 Remove configuration "hbase.replication"</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__class_hconstants_6"><a class="anchor" href="#__class_hconstants_6"></a>[−]class HConstants (6)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field DEFAULT_HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type boolean has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type java.lang.String has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field REPLICATION_ENABLE_DEFAULT of type boolean has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field REPLICATION_ENABLE_KEY of type java.lang.String has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field ZOOKEEPER_CONFIG_NAME of type java.lang.String has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Field ZOOKEEPER_USEMULTI of type java.lang.String has been removed from this class.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchFieldError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-18732: [compat 1-2] HBASE-14047 removed Cell methods without deprecation cycle.</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__interface_cell_5"><a class="anchor" href="#__interface_cell_5"></a>[−]interface Cell 5</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getFamily ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getMvccVersion ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getQualifier ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getRow ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method getValue ( ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-18795:Expose KeyValue.getBuffer() for tests alone. Allows KV#getBuffer in tests only that was deprecated previously.</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_region_scanner_changes"><a class="anchor" href="#_region_scanner_changes"></a>212.2.17. Region scanner changes:</h4>
<div class="sect4">
<h5 id="__interface_regionscanner_1"><a class="anchor" href="#__interface_regionscanner_1"></a>[−]interface RegionScanner (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Abstract method boolean nextRaw ( List&lt;Cell&gt;, int ) has been removed from this interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_storefile_changes"><a class="anchor" href="#_storefile_changes"></a>212.2.18. StoreFile changes:</h4>
<div class="sect4">
<h5 id="__class_storefile_1"><a class="anchor" href="#__class_storefile_1"></a>[−] class StoreFile (1)</h5>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">This class became interface.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception dependent on the usage of this class.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_mapreduce_changes"><a class="anchor" href="#_mapreduce_changes"></a>212.2.19. Mapreduce changes:</h4>
<div class="paragraph">
<p>HFile*Format has been removed in HBase 2.0.</p>
</div>
</div>
<div class="sect3">
<h4 id="_clusterstatus_changes"><a class="anchor" href="#_clusterstatus_changes"></a>212.2.20. ClusterStatus changes:</h4>
<div class="paragraph">
<p>HBASE-15843: Replace RegionState.getRegionInTransition() Map with a Set
hbase-client-1.0.0.jar, ClusterStatus.class package org.apache.hadoop.hbase</p>
</div>
<div class="sect4">
<h5 id="__clusterstatus_getregionsintransition_map_string_regionstate_1"><a class="anchor" href="#__clusterstatus_getregionsintransition_map_string_regionstate_1"></a>[−] ClusterStatus.getRegionsInTransition ( ) : Map&lt;String,RegionState&gt; 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/ClusterStatus.getRegionsInTransition:()Ljava/util/Map;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from java.util.Map&lt;java.lang.String,master.RegionState&gt; to java.util.List&lt;master.RegionState&gt;.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>Other changes in ClusterStatus include removal of convert methods that were no longer necessary after purge of PB from API.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="_purge_of_pbs_from_api"><a class="anchor" href="#_purge_of_pbs_from_api"></a>212.2.21. Purge of PBs from API</h4>
<div class="paragraph">
<p>PBs have been deprecated in APIs in HBase 2.0.</p>
</div>
<div class="sect4">
<h5 id="__hbasesnapshotexception_getsnapshotdescription_hbaseprotos_snapshotdescription_1"><a class="anchor" href="#__hbasesnapshotexception_getsnapshotdescription_hbaseprotos_snapshotdescription_1"></a>[−] HBaseSnapshotException.getSnapshotDescription ( ) : HBaseProtos.SnapshotDescription 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.getSnapshotDescription:()Lorg/apache/hadoop/hbase/protobuf/generated/HBaseProtos$SnapshotDescription;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription to org.apache.hadoop.hbase.client.SnapshotDescription.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="ulist">
<ul>
<li>
<p>HBASE-15609: Remove PB references from Result, DoubleColumnInterpreter and any such public facing class for 2.0.
hbase-client-1.0.0.jar, Result.class package org.apache.hadoop.hbase.client</p>
</li>
</ul>
</div>
</div>
<div class="sect4">
<h5 id="__result_getstats_clientprotos_regionloadstats_1"><a class="anchor" href="#__result_getstats_clientprotos_regionloadstats_1"></a>[−] Result.getStats ( ) : ClientProtos.RegionLoadStats 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/client/Result.getStats:()Lorg/apache/hadoop/hbase/protobuf/generated/ClientProtos$RegionLoadStats;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats to RegionLoadStats.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_rest_changes"><a class="anchor" href="#_rest_changes"></a>212.2.22. REST changes:</h4>
<div class="paragraph">
<p>hbase-rest-1.0.0.jar, Client.class package org.apache.hadoop.hbase.rest.client</p>
</div>
<div class="sect4">
<h5 id="__client_gethttpclient_httpclient_1"><a class="anchor" href="#__client_gethttpclient_httpclient_1"></a>[−] Client.getHttpClient ( ) : HttpClient 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/rest/client/Client.getHttpClient:()Lorg/apache/commons/httpclient/HttpClient</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.commons.httpclient.HttpClient to org.apache.http.client.HttpClient.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>hbase-rest-1.0.0.jar, Response.class package org.apache.hadoop.hbase.rest.client</p>
</div>
</div>
<div class="sect4">
<h5 id="__response_getheaders_header_1"><a class="anchor" href="#__response_getheaders_header_1"></a>[−] Response.getHeaders ( ) : Header[ ] 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/rest/client/Response.getHeaders:()[Lorg/apache/commons/httpclient/Header;</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from org.apache.commons.httpclient.Header[] to org.apache.http.Header[].</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_prettyprinter_changes"><a class="anchor" href="#_prettyprinter_changes"></a>212.2.23. PrettyPrinter changes:</h4>
<div class="paragraph">
<p>hbase-server-1.0.0.jar, HFilePrettyPrinter.class package org.apache.hadoop.hbase.io.hfile</p>
</div>
<div class="sect4">
<h5 id="__hfileprettyprinter_processfile_path_file_void_1"><a class="anchor" href="#__hfileprettyprinter_processfile_path_file_void_1"></a>[−]HFilePrettyPrinter.processFile ( Path file ) : void 1</h5>
<div class="paragraph">
<p>org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.processFile:(Lorg/apache/hadoop/fs/Path;)V</p>
</div>
<table class="tableblock frame-all grid-all spread">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Change</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Result</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Return value type has been changed from void to int.</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect3">
<h4 id="_accesscontrolclient_changes"><a class="anchor" href="#_accesscontrolclient_changes"></a>212.2.24. AccessControlClient changes:</h4>
<div class="paragraph">
<p>HBASE-13171 Change AccessControlClient methods to accept connection object to reduce setup time. Parameters have been changed in the following methods:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>hbase-client-1.2.7-SNAPSHOT.jar, AccessControlClient.class
package org.apache.hadoop.hbase.security.access
AccessControlClient.getUserPermissions ( Configuration conf, String tableRegex ) [static] : List&lt;UserPermission&gt; <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.getUserPermissions:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;)Ljava/util/List;</p>
</li>
<li>
<p>AccessControlClient.grant ( Configuration conf, String namespace, String userName, Permission.Action&#8230;&#8203; actions )[static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>AccessControlClient.grant ( Configuration conf, String userName, Permission.Action&#8230;&#8203; actions ) [static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>AccessControlClient.grant ( Configuration conf, TableName tableName, String userName, byte[ ] family, byte[ ] qual,Permission.Action&#8230;&#8203; actions ) [static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;[B[B[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>AccessControlClient.isAccessControllerRunning ( Configuration conf ) [static] : boolean <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.isAccessControllerRunning:(Lorg/apache/hadoop/conf/Configuration;)Z</p>
</li>
<li>
<p>AccessControlClient.revoke ( Configuration conf, String namespace, String userName, Permission.Action&#8230;&#8203; actions )[static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>AccessControlClient.revoke ( Configuration conf, String userName, Permission.Action&#8230;&#8203; actions ) [static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>AccessControlClient.revoke ( Configuration conf, TableName tableName, String username, byte[ ] family, byte[ ] qualifier,Permission.Action&#8230;&#8203; actions ) [static] : void <strong>DEPRECATED</strong>
org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;[B[B[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V</p>
</li>
<li>
<p>HBASE-18731: [compat 1-2] Mark protected methods of QuotaSettings that touch Protobuf internals as IA.Private</p>
</li>
</ul>
</div>
</div>
</div>
</div>
</div>
</div>
<div id="footnotes">
<hr>
<div class="footnote" id="_footnote_1">
<a href="#_footnoteref_1">1</a>. See 'Source Compatibility' <a href="https://blogs.oracle.com/darcy/entry/kinds_of_compatibility" class="bare">https://blogs.oracle.com/darcy/entry/kinds_of_compatibility</a>
</div>
<div class="footnote" id="_footnote_2">
<a href="#_footnoteref_2">2</a>. See <a href="http://docs.oracle.com/javase/specs/jls/se7/html/jls-13.html" class="bare">http://docs.oracle.com/javase/specs/jls/se7/html/jls-13.html</a>.
</div>
<div class="footnote" id="_footnote_3">
<a href="#_footnoteref_3">3</a>. Note that this indicates what could break, not that it will break. We will/should add specifics in our release notes.
</div>
<div class="footnote" id="_footnote_4">
<a href="#_footnoteref_4">4</a>. comp_matrix_offline_upgrade_note,Running an offline upgrade tool without downgrade might be needed. We will typically only support migrating data from major version X to major version X+1.
</div>
<div class="footnote" id="_footnote_5">
<a href="#_footnoteref_5">5</a>. The Metrics system was redone in HBase 0.96. See Migration to the New Metrics Hotness – Metrics2 by Elliot Clark for detail
</div>
</div>
<div id="footer">
<div id="footer-text">
Version 3.0.0-SNAPSHOT<br>
Last updated 2019-08-15 14:29:44 UTC
</div>
</div>
</body>
</html>