├── LICENSE ├── README.md └── hbase ├── BaseHbaseConnection.h ├── ConnectionPool.h ├── ConnectionPoolTest.cpp ├── Counters.h ├── HbcExtensions.h ├── Makefile.am ├── NativeHbaseClient-inl.cpp ├── NativeHbaseClient.cpp ├── NativeHbaseClient.h ├── configure.ac ├── hbase.thrift ├── hbc.cpp ├── m4 ├── ac_cxx_compile_stdcxx_0x.m4 └── ax_prefix_config.m4 ├── nhc-config.h └── testing └── NativeTesting.cpp /LICENSE: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | http://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Overview 2 | This is a work in progress! 3 | 4 | Dependencies: 5 | - HBase 0.94.0 or later 6 | - thrift 0.8.0 7 | - google logging 8 | - google flags 9 | - gcc 4.6 or later, or some similarly C++11 compliant compiler 10 | - facebook's folly library (get the latest from github at 11 | https://github.com/facebook/folly/) 12 | 13 | For now, see hbc.cpp and LiveClusterTesting.cpp for examples of use of 14 | the API. 15 | 16 | # Compiling 17 | 18 | First, setup and install folly as per the instructions for that 19 | project, as well as thrift and the other dependencies, either from 20 | source or from your operating system's package manager. Once that is 21 | complete, run 22 | 23 | cd hbase 24 | autoreconf --install 25 | D=/path/to/double-conversion-library 26 | LDFLAGS="-L$D/double-conversion" CFLAGS="-I$D/double-conversion/src" CXXFLAGS="-I$D/double-conversion/src" ./configure 27 | make 28 | make check 29 | make install 30 | 31 | Voila! You're done. 32 | 33 | # Setting up your HBase environment 34 | 35 | NHC currently requires HBase 0.94.0 or newer. In addition, you should 36 | configure your environment to enable the Region Server's embedded 37 | thrift server. Specifically, put the following in your hbase-site.xml file: 38 | 39 | 40 | hbase.regionserver.export.thrift 41 | true 42 | 43 | 44 | hbase.regionserver.thrift.server.type 45 | threadedselector 46 | 47 | 48 | hbase.regionserver.thrift.port 49 | 9091 50 | 51 | 52 | In a nutshell, this enables the in-region server thrift support, 53 | changes to the 'threadedselector' server type (the most efficient and 54 | high performance option), and asks it to listen to port 9091 (it will 55 | otherwise listen to port 9090 which conflicts with the default thrift 56 | port). 57 | 58 | Restart your cluster, and you should now be able to test! For example: 59 | 60 | ./hbc --zookeeper localhost:2181 tables 61 | 62 | To test the API with the hbc utility (included in this distribution). 63 | In fact... 64 | 65 | # hbc: a command line tool for working with hbase! 66 | hbc is a simple command line tool for use with an HBase cluster. The 67 | tool is capable of getting and setting data, creating and dropping 68 | tables, and a variety of other commonly useful tasks. 69 | 70 | ## Why not use hbase shell? 71 | 72 | There is nothing wrong with the hbase shell; it's quite useful and 73 | functional. But it is more complicated to use, requiring xml 74 | configuration files, and presents a somewhat non-Unix interface. hbc, 75 | on the other hand, requires one piece of configuration (the ZooKeeper 76 | cell the HBase cluster uses), and is well suited for more Unix-like 77 | interaction. It also is useful for testing the Native C++ Client and 78 | more suited for scripting. 79 | 80 | 81 | Usage: hbc [params] command [command params] 82 | 83 | Note: all commands rely on either the --zookeeper option or the 84 | HBC_ZOOKEEPER environment variable, which contains the ZooKeeper 85 | instance where the HBase cluster stores its state. 86 | 87 | No ZooKeeper instance set via command line or environment 88 | 89 | Subcommands: 90 | Table/Cell inspection: 91 | hbc tables 92 | hbc schema 93 | hbc regions
94 | 95 | Data manipulation: 96 | hbc scan [ --start_row R ] [ --end_row R ] [ --column C ] [--num_scan_rows N]
97 | hbc get
98 | hbc set
column=valuue column=value ... 99 | hbc delete
[column column ...] 100 | 101 | TAO operations: 102 | hbc taoput
103 | hbc taodelete
[true|false] 104 | hbc taoget
105 | 106 | Schema Manipulation: 107 | hbc create
/path/to/schema 108 | hbc create
(schema read from stdin) 109 | hbc drop
110 | hbc enable
111 | hbc disable
112 | hbc status
113 | 114 | ## Some Sample Usage 115 | [doctor@gallifrey hbase] hbc --zookeeper zk1:2181,zk2:2181 tables 116 | Table: abc_test_table 117 | Table: chip_mysql_import 118 | ... 119 | 120 | 121 | [doctor@gallifrey hbase] hbc --zookeeper zk1:2181,zk2:2181 scan --num_scan_rows 2 --start_row 400 chip_mysql_import 122 | Row 400 123 | fields:id - 15 124 | fields:value - "row 400" 125 | Row 4001 126 | fields:id - 324 127 | fields:value - "row 4001" 128 | Row 401 129 | fields:id - 17 130 | fields:value - "row 401" 131 | 132 | 133 | [doctor@gallifrey hbase] hbc --zookeeper zk1:2181,zk2:2181 schema chip_mysql_import | tee /tmp/table.schema 134 | # Table schema for chip_mysql_import 135 | # Dumped on Wed Jul 13 11:38:11 2011 136 | 137 | < 138 | name: fields: 139 | maxVersions: 3 140 | compression: LZO 141 | inMemory: 0 142 | bloomFilterType: NONE 143 | bloomFilterVectorSize: 0 144 | bloomFilterNbHashes: 0 145 | blockCacheEnabled: 1 146 | timeToLive: -1 147 | > 148 | [doctor@gallifrey hbase] hbc --zookeeper zk1:2181,zk2:2181 create chip_mysql_import_new < /tmp/table.schema 149 | Create successful! 150 | 151 | [doctor@gallifrey hbase] hbc --zookeeper zk1:2181,zk2:2181 schema chip_mysql_import_new 152 | # Table schema for chip_mysql_import_new 153 | # Dumped on Wed Jul 13 11:38:39 2011 154 | 155 | < 156 | name: rowdata: 157 | maxVersions: 3 158 | compression: LZO 159 | inMemory: 0 160 | bloomFilterType: NONE 161 | bloomFilterVectorSize: 0 162 | bloomFilterNbHashes: 0 163 | blockCacheEnabled: 1 164 | timeToLive: -1 165 | > 166 | 167 | [doctor@gallifrey hbase] ~/bin/hbc --zookeeper zk1:2181,zk2:2181 drop chip_mysql_import_new 168 | Unable to drop table: org.apache.hadoop.hbase.TableNotDisabledException: chip_mysql_import_new 169 | at org.apache.hadoop.hbase.master.HMaster.checkTableModifiable(HMaster.java:917) 170 | at org.apache.hadoop.hbase.master.handler.TableEventHandler.(TableEventHandler.java:54) 171 | at org.apache.hadoop.hbase.master.handler.DeleteTableHandler.(DeleteTableHandler.java:42) 172 | at org.apache.hadoop.hbase.master.HMaster.deleteTable(HMaster.java:834) 173 | at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source) 174 | at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) 175 | at java.lang.reflect.Method.invoke(Method.java:597) 176 | at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:569) 177 | at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1173) 178 | 179 | [doctor@gallifrey hbase] ~/bin/hbc --zookeeper zk1:2181,zk2:2181 disable chip_mysql_import_new 180 | Table disabled. 181 | [doctor@gallifrey hbase] ~/bin/hbc --zookeeper zk1:2181,zk2:2181 drop chip_mysql_import_new 182 | Table deleted. 183 | -------------------------------------------------------------------------------- /hbase/BaseHbaseConnection.h: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // This class represents a connection to an HBase server. It is 17 | // templated to be usable with different actual RPC classes (such as 18 | // the default thrift RPCs or the extended logstream APIs). 19 | 20 | #ifndef HBASE_SRC_BASEHBASECONNECTION_H 21 | #define HBASE_SRC_BASEHBASECONNECTION_H 22 | 23 | #include "hbase/nhc-config.h" 24 | 25 | #include 26 | #include 27 | 28 | #include "folly/Logging.h" 29 | 30 | #if !NHC_OPEN_SOURCE 31 | #include "protocol/THeaderProtocol.h" 32 | #include "transport/THeaderTransport.h" 33 | #endif 34 | 35 | #include "protocol/TBinaryProtocol.h" 36 | #include "protocol/TCompactProtocol.h" 37 | #include "transport/TBufferTransports.h" 38 | #include "transport/TSocket.h" 39 | 40 | namespace facebook { namespace hbase { 41 | 42 | #if !NHC_OPEN_SOURCE 43 | using apache::thrift::protocol::THeaderProtocol; 44 | using apache::thrift::transport::THeaderTransport; 45 | #endif 46 | 47 | using apache::thrift::protocol::TBinaryProtocol; 48 | using apache::thrift::protocol::TCompactProtocol; 49 | using apache::thrift::protocol::TProtocol; 50 | using apache::thrift::transport::TFramedTransport; 51 | using apache::thrift::transport::TBufferedTransport; 52 | using apache::thrift::transport::TSocket; 53 | using apache::thrift::transport::TTransport; 54 | using apache::thrift::TException; 55 | using std::string; 56 | 57 | /// Transport to use when connecting. 58 | enum HbaseClientTransport { 59 | DEFAULT_TRANSPORT, 60 | BUFFERED_TRANSPORT, 61 | FRAMED_TRANSPORT, 62 | COMPACT_FRAMED_TRANSPORT, 63 | HEADER_TRANSPORT, 64 | HEADER_TRANSPORT_WITH_ZLIB 65 | }; 66 | 67 | // Class representing a connection to a thrift server. Holds the 68 | // various Thrift classes such as the socket, transport, etc. 69 | template 70 | class BaseHbaseConnection : private boost::noncopyable { 71 | public: 72 | BaseHbaseConnection(const string& host, const int port); 73 | ~BaseHbaseConnection(); 74 | bool connect(HbaseClientTransport transport, 75 | int timeout_ms); 76 | T* thrift_client() { return client_.get(); } 77 | const string& host() const { return host_; } 78 | const int port() const { return port_; } 79 | void markUnhealthy() { ++errors_; } 80 | const bool isHealthy() const { return errors_ == 0; } 81 | 82 | private: 83 | const string host_; 84 | const int port_; 85 | std::unique_ptr client_; 86 | int errors_; 87 | }; 88 | 89 | template 90 | BaseHbaseConnection::BaseHbaseConnection( 91 | const string& host, const int port) 92 | : host_(host), port_(port), errors_(0) { 93 | } 94 | 95 | template 96 | bool BaseHbaseConnection::connect( 97 | HbaseClientTransport transport_type, 98 | int timeout_ms) { 99 | if (host_.empty()) { 100 | FB_LOG_EVERY_MS(ERROR, 1000) << "Request to connect to invalid host; " 101 | << "empty host specified?!"; 102 | return false; 103 | } 104 | if (port_ <= 0) { 105 | FB_LOG_EVERY_MS(ERROR, 1000) << "Invalid port for host " << host_; 106 | return false; 107 | } 108 | VLOG(1) << "Connecting to " << host_ << ":" << port_ << "..."; 109 | 110 | try { 111 | boost::shared_ptr socket(new TSocket(host_, port_)); 112 | if (timeout_ms > 0) { 113 | socket->setConnTimeout(timeout_ms); 114 | socket->setRecvTimeout(timeout_ms); 115 | socket->setSendTimeout(timeout_ms); 116 | } 117 | 118 | assert(transport_type != DEFAULT_TRANSPORT); 119 | 120 | boost::shared_ptr protocol; 121 | boost::shared_ptr transport; 122 | if (transport_type == BUFFERED_TRANSPORT) { 123 | transport.reset(new TBufferedTransport(socket)); 124 | protocol.reset(new TBinaryProtocol(transport)); 125 | } else if (transport_type == FRAMED_TRANSPORT) { 126 | transport.reset(new TFramedTransport(socket)); 127 | protocol.reset(new TBinaryProtocol(transport)); 128 | } else if (transport_type == COMPACT_FRAMED_TRANSPORT) { 129 | transport.reset(new TFramedTransport(socket)); 130 | protocol.reset(new TCompactProtocol(transport)); 131 | } 132 | #if !NHC_OPEN_SOURCE 133 | else if (transport_type == HEADER_TRANSPORT) { 134 | transport.reset(new THeaderTransport(socket)); 135 | protocol.reset(new THeaderProtocol(transport)); 136 | } else if (transport_type == HEADER_TRANSPORT_WITH_ZLIB) { 137 | auto transport_tmp = new THeaderTransport(socket); 138 | transport_tmp->setTransform(THeaderTransport::ZLIB_TRANSFORM); 139 | transport.reset(transport_tmp); 140 | protocol.reset(new THeaderProtocol(transport)); 141 | } 142 | #endif 143 | 144 | client_.reset(new T(protocol)); 145 | transport->open(); 146 | VLOG(1) << "done"; 147 | } 148 | catch (const TException& e) { 149 | LOG(ERROR) << "Unable to connect to " << host_ << ":" << port_ << ": " 150 | << e.what(); 151 | return false; 152 | } 153 | return true; 154 | } 155 | 156 | template 157 | BaseHbaseConnection::~BaseHbaseConnection() { 158 | } 159 | 160 | } } // namespace 161 | 162 | #endif // HBASE_SRC_BASEHBASECONNECTION_H 163 | -------------------------------------------------------------------------------- /hbase/ConnectionPool.h: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // A fairly simple, highly templetized connection pool, intended for 17 | // use with the HBase Client. 18 | // 19 | // A connection pool is similar to a hash, and is templetized on two 20 | // types -- the key to the hash, and the entries. However, in 21 | // addition to this, the pool knows how to create new entries for the 22 | // pool (via callbacks). This allows the pool to expand as needed. 23 | // See the test case for basic usage. 24 | 25 | #ifndef HBASE_SRC_CONNECTIONPOOL_H 26 | #define HBASE_SRC_CONNECTIONPOOL_H 27 | 28 | #include 29 | #include 30 | #include 31 | #include 32 | 33 | #include "folly/Logging.h" 34 | 35 | #include "hbase/Counters.h" 36 | 37 | // This needs to be declared before the template below for clang (ADL 38 | // rules, see http://clang.llvm.org/compatibility.html, somewhat 39 | // complicated by 'const std::pair' being the 40 | // typedef target of HostMapKey). 41 | std::ostream& operator<<(std::ostream& out, 42 | const std::pair& key); 43 | 44 | namespace facebook { namespace hbase { 45 | using std::unordered_multimap; 46 | 47 | template 48 | class ConnectionPool : private boost::noncopyable { 49 | public: 50 | // Types for our callbacks and map. 51 | typedef unordered_multimap connection_multimap; 52 | typedef std::function ConnectionFactory; 53 | typedef std::function ConnectionKeyMaker; 54 | 55 | // The conneciton pool is templetized on the Key and Conn types. In 56 | // addition, the pool contains a maximum number of retained 57 | // connections per key (if more than this limit are requested, they 58 | // will be created on the fly, and deleted when they are finished 59 | // being used). 60 | ConnectionPool(int max_retained_connections_per_key, 61 | int max_total_connections_per_key, 62 | ConnectionFactory factory, 63 | ConnectionKeyMaker key_maker, 64 | CounterBase* counters = NULL) 65 | : max_retained_connections_per_key_(max_retained_connections_per_key), 66 | max_total_connections_per_key_(max_total_connections_per_key), 67 | connection_factory_(factory), 68 | connection_key_maker_(key_maker), 69 | stats_counters_(counters) { 70 | if (max_total_connections_per_key_ > 0) { 71 | CHECK_GE(max_total_connections_per_key_, 72 | max_retained_connections_per_key_); 73 | } 74 | } 75 | 76 | // We should never be destroyed if there are any in-use connections. 77 | ~ConnectionPool() { 78 | std::lock_guard g(connections_mutex_); 79 | VLOG(1) << "Clearing out " << available_connections_.size() 80 | << " entries in the connection pool."; 81 | CHECK_EQ(in_use_connections_.size(), 0) 82 | << "Cannot destroy a connection pool while there are in-use " 83 | << "connections!"; 84 | }; 85 | 86 | // Given a key, find a connection in our unused connections, or 87 | // create a new connection. Note there is a race here; if multiple 88 | // threads decide to invoke connection_factory_ for this key, then 89 | // multiple entries will be simultaneously created. That's okay, 90 | // the worst result is we missed a chance to re-use an entry 91 | // returned to the pool. 92 | Conn* lookup(const Key& key) { 93 | std::unique_ptr ret; 94 | { 95 | std::lock_guard g(connections_mutex_); 96 | auto p = available_connections_.find(key); 97 | if (p != available_connections_.end()) { 98 | if (stats_counters_) { 99 | stats_counters_->incrementCounter("reused_connections"); 100 | } 101 | ret = std::move(p->second); 102 | available_connections_.erase(p); 103 | VLOG(1) << "Using already created connection for " << key; 104 | } 105 | if (!ret.get()) { 106 | int total_key_connections = available_connections_.count(key) + 107 | in_use_connections_.count(key); 108 | VLOG(1) << "Total outstanding connections for " << key << ": " 109 | << total_key_connections; 110 | if (max_total_connections_per_key_ > 0 && 111 | total_key_connections >= max_total_connections_per_key_) { 112 | FB_LOG_EVERY_MS(ERROR, 500) << "Hit max total allowed connections " 113 | << "for " << key; 114 | return NULL; 115 | } 116 | } 117 | } 118 | if (!ret.get()) { 119 | ret.reset(connection_factory_(key)); 120 | if (stats_counters_) { 121 | stats_counters_->incrementCounter("created_connections"); 122 | } 123 | if (!ret.get()) { 124 | LOG(ERROR) << "ConnectionPool factory failed to create connection" 125 | << " for key:" << key; 126 | return NULL; 127 | } 128 | VLOG(1) << "Creating connection for " << key; 129 | } 130 | std::lock_guard g(connections_mutex_); 131 | Conn* ret_ptr = ret.get(); 132 | in_use_connections_.insert(make_pair(key, std::move(ret))); 133 | return ret_ptr; 134 | } 135 | 136 | // Release a key. If this would put us under the limit we have per 137 | // key, retain the connection, otherwise go ahead and free it. 138 | void release(Conn* conn) { 139 | CHECK(conn != NULL); 140 | if (stats_counters_) { 141 | stats_counters_->incrementCounter("released_connections"); 142 | } 143 | 144 | std::unique_ptr conn_holder; 145 | const Key key(connection_key_maker_(conn)); 146 | std::lock_guard g(connections_mutex_); 147 | auto p = in_use_connections_.equal_range(key); 148 | for (auto it = p.first; it != p.second; ++it) { 149 | if ((*it).second.get() == conn) { 150 | conn_holder = std::move((*it).second); 151 | in_use_connections_.erase(it); 152 | break; 153 | } 154 | } 155 | 156 | if (available_connections_.count(key) + in_use_connections_.count(key) < 157 | max_retained_connections_per_key_) { 158 | if (conn->isHealthy()) { 159 | VLOG(1) << "Returning healthy connection " << conn << " for " << key 160 | << " to pool"; 161 | // Let's double check the exact same connection we're 162 | // releasing isn't already in available_connections_. Just in 163 | // case. 164 | auto p = available_connections_.equal_range(key); 165 | for (auto it = p.first; it != p.second; ++it) { 166 | if ((*it).second.get() == conn) { 167 | LOG(ERROR) << "Attempt to double release a connection?! " 168 | << "Shocking you didn't already crash."; 169 | return; 170 | } 171 | } 172 | 173 | available_connections_.insert(make_pair(key, std::move(conn_holder))); 174 | } else { 175 | if (stats_counters_) { 176 | stats_counters_->incrementCounter("discarded_connections"); 177 | } 178 | LOG(ERROR) << "Discarding unhealthy connection " << conn << " for " 179 | << key; 180 | } 181 | } else { 182 | VLOG(1) << "Too many connections for " << key << ", deleting " 183 | << conn; 184 | } 185 | } 186 | 187 | private: 188 | const int max_retained_connections_per_key_; 189 | const int max_total_connections_per_key_; 190 | const ConnectionFactory connection_factory_; 191 | const ConnectionKeyMaker connection_key_maker_; 192 | 193 | std::mutex connections_mutex_; 194 | unordered_multimap> available_connections_; 195 | unordered_multimap> in_use_connections_; 196 | 197 | CounterBase* stats_counters_; // unowned 198 | }; 199 | 200 | } } // facebook::hbase 201 | 202 | #endif // HBASE_SRC_CONNECTIONPOOL_H 203 | -------------------------------------------------------------------------------- /hbase/ConnectionPoolTest.cpp: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2012 Facebook, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | // Test case for the ConnectionPool class. 18 | 19 | #include 20 | #include 21 | 22 | #include "hbase/ConnectionPool.h" 23 | 24 | #include 25 | 26 | namespace h = facebook::hbase; 27 | using std::string; 28 | using std::unordered_set; 29 | 30 | class TestPoolEntry { 31 | public: 32 | explicit TestPoolEntry(const string& s) : entry_name(s), healthy(true) { 33 | all_entries.insert(this); 34 | } 35 | ~TestPoolEntry() { 36 | all_entries.erase(this); 37 | } 38 | bool isHealthy() const { return healthy; } 39 | 40 | const string entry_name; 41 | static unordered_set all_entries; 42 | bool healthy; 43 | }; 44 | 45 | unordered_set TestPoolEntry::all_entries; 46 | 47 | TestPoolEntry* factory(const string& s) { 48 | return new TestPoolEntry(s); 49 | } 50 | 51 | TestPoolEntry* null_factory(const string& s) { 52 | return NULL; 53 | } 54 | 55 | const string key_maker(const TestPoolEntry* entry) { 56 | return entry->entry_name; 57 | } 58 | 59 | TEST(ConnectionPool, ConnectionNullPoolTest) { 60 | h::ConnectionPool test_pool( 61 | 100, 0, null_factory, key_maker); 62 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 63 | TestPoolEntry* e1 = test_pool.lookup("test"); 64 | EXPECT_TRUE(e1 == NULL); 65 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 66 | } 67 | 68 | TEST(ConnectionPool, ConnectionErrorTest) { 69 | h::ConnectionPool test_pool( 70 | 100, 0, factory, key_maker); 71 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 72 | TestPoolEntry* e1 = test_pool.lookup("test"); 73 | EXPECT_TRUE(e1 != NULL); 74 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 75 | test_pool.release(e1); 76 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 77 | e1 = test_pool.lookup("test"); 78 | e1->healthy = false; 79 | test_pool.release(e1); 80 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 81 | } 82 | 83 | TEST(ConnectionPool, ConnectionPoolTest) { 84 | { 85 | // Some basic tests with a "large" pool. 86 | h::ConnectionPool test_pool(100, 0, factory, 87 | key_maker); 88 | // starts empty... 89 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 90 | // should instantiate a new one by calling our factory method, 91 | // which we then let go, but confirm it still exists in the pool. 92 | TestPoolEntry* e1 = test_pool.lookup("test"); 93 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 94 | test_pool.release(e1); 95 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 96 | // if we ask for one again, we get our old friend e1 back! 97 | EXPECT_EQ(e1, test_pool.lookup("test")); 98 | test_pool.release(e1); 99 | // confirm adding a second one doesn't give us the same entry 100 | TestPoolEntry* e2 = test_pool.lookup("test"); 101 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 102 | TestPoolEntry* e3 = test_pool.lookup("test"); 103 | EXPECT_NE(e2, e3); 104 | test_pool.release(e2); 105 | test_pool.release(e3); 106 | } 107 | // confirm ConnectionPool destructor empties out all our pool entries 108 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 109 | } 110 | 111 | TEST(ConnectionPool, ConnectionPoolLimitTest) { 112 | // Test a pool with a limit of two retained connections. First 113 | // check the basic stuff. 114 | h::ConnectionPool test_pool(2, 100, factory, 115 | key_maker); 116 | EXPECT_EQ(0, TestPoolEntry::all_entries.size()); 117 | TestPoolEntry* e1 = test_pool.lookup("test"); 118 | EXPECT_EQ(1, TestPoolEntry::all_entries.size()); 119 | 120 | // Instantiate a "lot" of pool entries. All but 2 will be freed as 121 | // soon as they are released, as per the construction to test_pool. 122 | std::vector entries; 123 | for (int i = 0; i < 500; ++i) { 124 | TestPoolEntry* e = test_pool.lookup("test"); 125 | // e1 plus the first 99 of this loop should get us to the 100 hard 126 | // limit; so subsequent calls will return NULL. Verify that here. 127 | if (i >= 99) { 128 | EXPECT_TRUE(e == NULL); 129 | } else { 130 | EXPECT_TRUE(e != NULL); 131 | entries.push_back(e); 132 | } 133 | } 134 | 135 | // All of 'entries' plus e1. Since we tried to create 500 entries 136 | // with a max retain count of 100, 400 were null so all_entries 137 | // should be limited in size. 138 | EXPECT_EQ(entries.size() + 1, TestPoolEntry::all_entries.size()); 139 | 140 | // Release them all; should go back down to 2 entries in the pool 141 | // (e1 and a now unreferenced entry that was in the 'entries' 142 | // vector). 143 | for (TestPoolEntry* entry : entries) { 144 | test_pool.release(entry); 145 | } 146 | EXPECT_EQ(2, TestPoolEntry::all_entries.size()); 147 | 148 | test_pool.release(e1); 149 | } 150 | 151 | int main(int argc, char *argv[]) { 152 | testing::InitGoogleTest(&argc, argv); 153 | google::ParseCommandLineFlags(&argc, &argv, true); 154 | return RUN_ALL_TESTS(); 155 | } 156 | -------------------------------------------------------------------------------- /hbase/Counters.h: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // A basic definition of what a counter class is, and a trivial, 17 | // non-performant implementation using a simple hash map to store 18 | // counters. More sophisticated implementations might export stats to 19 | // a thrift server or other external source. 20 | 21 | #ifndef HBASE_SRC_COUNTERS_H 22 | #define HBASE_SRC_COUNTERS_H 23 | 24 | #include 25 | #include 26 | #include 27 | #include 28 | #include 29 | 30 | #include "boost/noncopyable.hpp" 31 | 32 | #include "folly/String.h" 33 | #include "folly/Histogram.h" 34 | 35 | namespace facebook { namespace hbase { 36 | 37 | using std::map; 38 | using std::string; 39 | using std::unordered_map; 40 | 41 | typedef folly::Histogram CounterHistogram; 42 | 43 | class CounterBase : private boost::noncopyable { 44 | public: 45 | const string fieldName(const string& field) { 46 | if (prefix_.empty()) { 47 | return field; 48 | } else { 49 | return prefix_ + "_" + field; 50 | } 51 | } 52 | 53 | // Counters may or may not have prefixes; if specified, the prefix 54 | // is prepended to every value when snapshotted and should *not* be 55 | // part of the "counter" to increment via incrementCounter. 56 | explicit CounterBase(const string& prefix) : prefix_(prefix) { } 57 | CounterBase() { } 58 | 59 | virtual ~CounterBase() { } 60 | virtual void incrementCounter(const string& counter, int64_t incr = 1) { } 61 | virtual void addHistogramValue(const string& counter, int64_t value) { } 62 | 63 | virtual void snapshot(map* output, 64 | map* output_buckets) { } 65 | 66 | private: 67 | const string prefix_; 68 | }; 69 | 70 | // A SimpleCounter is a very simple, inefficient counter for tracking 71 | // stats. 72 | class SimpleCounter : public CounterBase { 73 | public: 74 | explicit SimpleCounter(const string& prefix) : CounterBase(prefix) { init(); } 75 | SimpleCounter() : CounterBase() { init(); } 76 | 77 | void init() { 78 | addHistogram("rowop_rpc_time_micros", 20*1000, 0, 1000*1000); 79 | addHistogram("root_rpc_time_micros", 20*1000, 0, 1000*1000); 80 | } 81 | 82 | void addHistogram(const string& key, int bucket_width, int lower, int upper) { 83 | histogram_stats_[key].reset(new CounterHistogram(bucket_width, 84 | lower, upper)); 85 | } 86 | 87 | virtual void addHistogramValue(const string& counter, int64_t value) { 88 | std::lock_guard lock(mutex_); 89 | auto it = histogram_stats_.find(counter); 90 | if (it != histogram_stats_.end()) { 91 | it->second->addValue(value); 92 | } 93 | } 94 | 95 | virtual ~SimpleCounter() { } 96 | virtual void incrementCounter(const string& counter, int64_t incr = 1) { 97 | std::lock_guard lock(mutex_); 98 | stats_[counter] += incr; 99 | } 100 | 101 | virtual void snapshot(map* output, 102 | map* output_buckets) { 103 | std::lock_guard lock(mutex_); 104 | 105 | output->clear(); 106 | output_buckets->clear(); 107 | for (auto& kv : stats_) { 108 | output->insert(make_pair(fieldName(kv.first), kv.second)); 109 | } 110 | for (auto& kv : histogram_stats_) { 111 | const string field_name = fieldName(kv.first); 112 | CounterHistogram *h = kv.second.get(); 113 | 114 | // Write histogram percentile information to output, not 115 | // output_buckets, since it is simply numeric data. 116 | output->insert(make_pair(field_name + ".p50", 117 | h->getPercentileEstimate(0.50))); 118 | output->insert(make_pair(field_name + ".p95", 119 | h->getPercentileEstimate(0.95))); 120 | output->insert(make_pair(field_name + ".p99", 121 | h->getPercentileEstimate(0.99))); 122 | 123 | string bucket_string; 124 | 125 | for (int i = 0; i < h->getNumBuckets(); ++i) { 126 | if (i > 0) bucket_string += ","; 127 | const auto& bucket = h->getBucketByIndex(i); 128 | double average = 0.0; 129 | if (bucket.count > 0) 130 | average = bucket.sum / bucket.count; 131 | 132 | int64_t bucket_min = std::numeric_limits::min(); 133 | if (i > 0) { 134 | bucket_min = h->getMin() + (i - 1) * h->getBucketSize(); 135 | } 136 | folly::stringAppendf(&bucket_string, "%ld:%ld:%ld", 137 | bucket_min, 138 | bucket.count, 139 | (int64_t)average); 140 | } 141 | output_buckets->insert(make_pair(field_name, bucket_string)); 142 | } 143 | } 144 | 145 | private: 146 | mutable std::mutex mutex_; 147 | unordered_map stats_; 148 | unordered_map > histogram_stats_; 149 | }; 150 | 151 | } } // facebook::hbase 152 | 153 | #endif // HBASE_SRC_COUNTERS_H 154 | -------------------------------------------------------------------------------- /hbase/HbcExtensions.h: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // This file describes the interface for extending hbc. It contains 17 | // the necessary classes for defining commands and command groups as 18 | // well as registering special printers and alternative connection 19 | // classes (for using derivations of NativeHbaseClient). 20 | 21 | #ifndef HBASE_SRC_HBCEXTENSIONS_H 22 | #define HBASE_SRC_HBCEXTENSIONS_H 23 | 24 | #include 25 | #include 26 | 27 | #include "boost/noncopyable.hpp" 28 | 29 | #include "hbase/NativeHbaseClient.h" 30 | 31 | namespace facebook { namespace hbase { 32 | 33 | namespace h = facebook::hbase; 34 | using std::string; 35 | 36 | // A simple set of classes to handle subcommands, grouped by theme. 37 | // Each subcommand such as "get" or "scan" is an instance of a 38 | // subclass of Subcommand. Whenever an instance of Subcommand is 39 | // created, it adds itself to the most recently created command 40 | // group's commands vector, which is used for iteration over all 41 | // commands (such as for usage or to dispatch the command). 42 | class CommandGroup; 43 | 44 | // Thrown during execution when there is an issue with the supplied 45 | // arguments. 46 | class UsageException : public std::exception { }; 47 | 48 | // A simple exception that indicates something went wrong that is not 49 | // recoverable. The intention is for the message to be printed (with 50 | // nothing else) and the process terminate. 51 | class FatalException : public std::exception { 52 | public: 53 | explicit FatalException(const string& s) : what_(s) { } 54 | virtual ~FatalException() throw() { } 55 | virtual const char* what() const throw() { 56 | return what_.c_str(); 57 | } 58 | private: 59 | const string what_; 60 | }; 61 | 62 | class Subcommand; 63 | class CommandGroup : boost::noncopyable { 64 | public: 65 | explicit CommandGroup(const string& name_in) : name(name_in) { } 66 | void addSubcommand(Subcommand* subcommand) { 67 | commands.push_back(subcommand); 68 | } 69 | 70 | const string name; 71 | vector commands; 72 | }; 73 | 74 | // Subcommand itself; very simple, just two virtual methods and a 75 | // string name. Each subcommand will subclass this class. 76 | class Subcommand : boost::noncopyable { 77 | public: 78 | explicit Subcommand(const string& name_in) : name(name_in) { } 79 | 80 | const string extractTable(h::NativeHbaseClient* client, 81 | const vector& args, int pos) { 82 | if (pos >= args.size()) { 83 | throw FatalException("Table not specified"); 84 | } 85 | 86 | const string &table = args[pos]; 87 | vector all_tables; 88 | client->listTables(&all_tables); 89 | auto it = find(all_tables.begin(), all_tables.end(), table); 90 | if (it == all_tables.end()) { 91 | throw FatalException("Table does not exist: " + table); 92 | } 93 | 94 | return table; 95 | } 96 | 97 | virtual ~Subcommand() { } 98 | 99 | virtual void Usage(const string& prefix) = 0; 100 | virtual int Execute(h::NativeHbaseClient* client, 101 | const vector& args) = 0; 102 | 103 | const string name; 104 | }; 105 | 106 | typedef std::function ConnectionMaker; 107 | typedef std::function RowPrinter; 108 | void RegisterExtensionCommands(CommandGroup* group); 109 | void RegisterConnectionMaker(ConnectionMaker maker); 110 | void RegisterSpecialRowPrinter(RowPrinter printer); 111 | 112 | } } // facebook::hbase 113 | 114 | #endif // HBASE_SRC_HBCEXTENSIONS_H 115 | -------------------------------------------------------------------------------- /hbase/Makefile.am: -------------------------------------------------------------------------------- 1 | SUBDIRS = . 2 | 3 | ACLOCAL_AMFLAGS = -I m4 4 | 5 | BUILT_SOURCES = gen-cpp/hbase/Hbase.cpp 6 | 7 | gen-cpp/hbase/Hbase.cpp: hbase.thrift 8 | mkdir -p gen-cpp/hbase 9 | $(THRIFT_EXEC_PATH) --gen cpp -out gen-cpp/hbase $< 10 | 11 | gen-cpp/hbase/Hbase.h: hbase.thrift 12 | mkdir -p gen-cpp/hbase 13 | $(THRIFT_EXEC_PATH) --gen cpp -out gen-cpp/hbase $< 14 | 15 | bin_PROGRAMS = hbc 16 | lib_LTLIBRARIES = libhbase.la 17 | noinst_PROGRAMS = connection_pool_test native_testing 18 | 19 | libhbase_includedir = $(includedir)/hbase 20 | libhbase_include_HEADERS = BaseHbaseConnection.h ConnectionPool.h Counters.h HbcExtensions.h NativeHbaseClient.h NativeHbaseClient-inl.cpp \ 21 | gen-cpp/hbase/hbase_constants.h gen-cpp/hbase/Hbase.h gen-cpp/hbase/hbase_types.h nhc-config.h 22 | libhbase_la_SOURCES = NativeHbaseClient.cpp \ 23 | gen-cpp/hbase/hbase_constants.cpp gen-cpp/hbase/Hbase.cpp gen-cpp/hbase/hbase_types.cpp 24 | libhbase_la_LIBADD = -lfolly -lthrift 25 | libhbase_la_CPPFLAGS = -Igen-cpp $(THRIFT_CFLAGS) 26 | 27 | hbc_SOURCES = hbc.cpp 28 | hbc_LDADD = -lhbase 29 | hbc_CPPFLAGS = -Igen-cpp $(THRIFT_CFLAGS) 30 | 31 | TESTS = connection_pool_test 32 | 33 | connection_pool_test_SOURCES = ConnectionPoolTest.cpp 34 | connection_pool_test_LDADD = -lfolly 35 | 36 | native_testing_SOURCES = testing/NativeTesting.cpp 37 | native_testing_LDADD = -lhbase 38 | native_testing_CPPFLAGS = $(THRIFT_CFLAGS) -Igen-cpp 39 | 40 | clean-local: 41 | rm -rf gen-cpp 42 | -------------------------------------------------------------------------------- /hbase/NativeHbaseClient-inl.cpp: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // In-line template declarations. Included in a couple of headers as 17 | // a hack to get around a gcc 4.6.2 segfault. 18 | 19 | #ifndef HBASE_SRC_NATIVEHBASECLIENT_INL_H_ 20 | #define HBASE_SRC_NATIVEHBASECLIENT_INL_H_ 21 | 22 | namespace facebook { namespace hbase { 23 | 24 | // A generic wrapper for thrift operations; if a Thrift protocol error 25 | // occurs, the connection is marked unhealthy. 26 | template 27 | T invokeThriftOperation(NativeHbaseClient* client, 28 | NativeHbaseConnection* conn, 29 | typename ThriftOperation::Type op) { 30 | client->getCounters()->incrementCounter("regionserver_operations"); 31 | if (!conn->isHealthy()) { 32 | throw NativeHbaseException("Invoked operation on unhealthy connection"); 33 | } 34 | try { 35 | return op(conn->thrift_client()); 36 | } 37 | catch (const TTransportException& te) { 38 | conn->markUnhealthy(); 39 | LOG(ERROR) << "Transport Exception received, retrying." 40 | << " Error detail:" << te.what() 41 | << " Server:" << conn->host() << ":" << conn->port(); 42 | throw; 43 | } 44 | } 45 | 46 | // A wrapper for operations on the connection to the root server. We 47 | // attempt one retry; if it fails, we mark ourselves unhealthy and 48 | // propagate the exception. 49 | template 50 | T NativeHbaseClient::invokeRootOperation(NativeHbaseClient* client, 51 | const char* opname, 52 | typename ThriftOperation::Type op) { 53 | stats_counters_->incrementCounter("root_operations"); 54 | std::lock_guard g(root_conn_mutex_); 55 | if (!client->isHealthy() || !client->root_conn_->isHealthy()) { 56 | stats_counters_->incrementCounter("unhealthy_root_operations"); 57 | throw NativeHbaseException( 58 | "Invoked operation on unhealthy NativeHbaseClient"); 59 | } 60 | for (int attempt = 0; attempt < 2; ++attempt) { 61 | try { 62 | auto now = system_clock::now(); 63 | ScopeGuard g = folly::makeGuard( 64 | [=] { 65 | stats_counters_->addHistogramValue( 66 | "root_rpc_time_micros", 67 | microseconds(system_clock::now() - now).count()); 68 | } 69 | ); 70 | logHbaseOperation(opname, 71 | client->root_conn_->host(), client->root_conn_->port(), 72 | "", timeout_ms_); 73 | return op(client->root_conn_->thrift_client()); 74 | } 75 | catch (const TTransportException& te) { 76 | stats_counters_->incrementCounter("root_operation_failures"); 77 | const string error_message = 78 | string("Transport Exception received when communicating to root region " 79 | "server: ") + te.what(); 80 | logHbaseError(opname, 81 | client->root_conn_->host(), client->root_conn_->port(), 82 | "", timeout_ms_, error_message); 83 | LOG(ERROR) << error_message; 84 | if (attempt == 0) { 85 | LOG(ERROR) << "Retrying once"; 86 | if (!initRootConnection()) { 87 | LOG(ERROR) << "Re-initializing root connection failed, aborting"; 88 | logHbaseError( 89 | opname, client->root_conn_->host(), client->root_conn_->port(), 90 | "", timeout_ms_, "Error retrying root operation, aborting"); 91 | throw; 92 | } 93 | continue; 94 | } 95 | client->root_conn_->markUnhealthy(); 96 | healthy_ = false; 97 | throw; 98 | } 99 | catch(...) { 100 | throw; 101 | } 102 | } 103 | CHECK(false) << "should never reach this"; 104 | } 105 | 106 | // A generic wrapper for interacting with the Thrift client objects. 107 | // We retry some operations, possibly re-reading META to locate a 108 | // region server (in the case of an IOError that looks like "Region is 109 | // not online:"). For TTransportExceptions, we simply re-execute the 110 | // operation once after marking it unhealthy (which ensures that, when 111 | // the connection is released, it is not reused and instead we 112 | // recreate a connection). All other errors are reported and raised 113 | // without retrying. 114 | template 115 | T NativeHbaseClient::invokeRowOperation(const string& table, const string& row, 116 | const char* opname, 117 | typename ThriftOperation::Type op) { 118 | bool ignore_cache = false; 119 | for (int attempt = 0; attempt < 2; ++attempt) { 120 | string host; 121 | int port = -1; 122 | if (attempt > 0) { 123 | stats_counters_->incrementCounter("rowop_region_server_retry"); 124 | } 125 | try { 126 | auto now = system_clock::now(); 127 | ScopeGuard g = folly::makeGuard( 128 | [=] { 129 | stats_counters_->addHistogramValue( 130 | "rowop_rpc_time_micros", 131 | microseconds(system_clock::now() - now).count()); 132 | } 133 | ); 134 | NativeHbaseConnection* conn = findConnection(table, row, ignore_cache); 135 | host = conn->host(); 136 | port = conn->port(); 137 | ScopeGuard g2 = folly::makeGuard( 138 | [=] { this->connection_pool_.release(conn); } ); 139 | logHbaseOperation(opname, host, port, "", timeout_ms_); 140 | return invokeThriftOperation(this, conn, op); 141 | } 142 | catch (const TTransportException& te) { 143 | stats_counters_->incrementCounter("rowop_region_server_exception"); 144 | // invokeThriftOperation will mark the connection unhealthy so 145 | // that when we retry, we get a new one; so just continue to the 146 | // retry on the first attempt. We need to ignore the cache on 147 | // the next read in case the region server is down -- we want to 148 | // re-query the master to find the region. 149 | const string error_message = 150 | to("Region server transport exception, table: ", table, 151 | ", row: ", humanify(row), ", exception: ", te.what()); 152 | FB_LOG_EVERY_MS(ERROR, 100) << error_message; 153 | logHbaseError(opname, host, port, row, timeout_ms_, error_message); 154 | if (attempt == 0) { 155 | ignore_cache = true; 156 | continue; 157 | } 158 | throw; 159 | } 160 | catch (const IOError& e) { 161 | stats_counters_->incrementCounter("rowop_region_server_exception"); 162 | const string error_message = 163 | to("Region server io exception, table: ", table, 164 | ", row: ", humanify(row), ", exception: ", e.what()); 165 | FB_LOG_EVERY_MS(ERROR, 100) << error_message; 166 | logHbaseError(opname, host, port, row, timeout_ms_, error_message); 167 | if (attempt == 0 && 168 | boost::starts_with(e.message, "Region is not online:")) { 169 | LOG(ERROR) << "Retrying operation..."; 170 | ignore_cache = true; 171 | continue; 172 | } 173 | throw; 174 | } 175 | catch (const std::exception& e) { 176 | stats_counters_->incrementCounter("rowop_region_server_exception"); 177 | const string error_message = 178 | to("Region server unknown exception, table: ", table, 179 | ", row: ", humanify(row), ", exception: ", e.what()); 180 | FB_LOG_EVERY_MS(ERROR, 100) << error_message; 181 | logHbaseError(opname, host, port, row, timeout_ms_, error_message); 182 | throw; 183 | } 184 | catch(...) { 185 | stats_counters_->incrementCounter("rowop_region_server_exception"); 186 | const string error_message = 187 | to("Unknown type of exception during RPC, table: ", table, 188 | ", row: ", humanify(row)); 189 | FB_LOG_EVERY_MS(ERROR, 100) << error_message; 190 | logHbaseError(opname, host, port, row, timeout_ms_, error_message); 191 | throw; 192 | } 193 | } 194 | CHECK(0); // should never get here 195 | return T(); 196 | } 197 | 198 | } } // namespaces 199 | 200 | #endif // HBASE_SRC_NATIVEHBASECLIENT_INL_H_ 201 | -------------------------------------------------------------------------------- /hbase/NativeHbaseClient.cpp: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | #include "hbase/NativeHbaseClient.h" 17 | #include "hbase/NativeHbaseClient-inl.cpp" 18 | 19 | #include 20 | #include 21 | #include 22 | #include 23 | #include 24 | 25 | #include 26 | 27 | #include "folly/Bits.h" 28 | #include "folly/Logging.h" 29 | #include "folly/Conv.h" 30 | #include "folly/String.h" 31 | #include "folly/ScopeGuard.h" 32 | #include "folly/String.h" 33 | 34 | using namespace std; 35 | using apache::hadoop::hbase::thrift::IOError; 36 | using apache::hadoop::hbase::thrift::IllegalArgument; 37 | using folly::Endian; 38 | using folly::makeGuard; 39 | 40 | DEFINE_int32(hbase_port, 9091, "Port to use when connecting to Hbase servers"); 41 | DEFINE_int32(hbase_metadata_rows_per_fetch, 65536, 42 | "default number of rows to fetch per RPC for hbase metadata"); 43 | DEFINE_int32(thrift_proxy_port, 9090, "Port the HBase Thrift Proxy runs on"); 44 | DEFINE_bool(force_thrift_proxy_mode, false, "if set, communicate with standard " 45 | "thrift proxies rather than zookeeper and region servers. Do " 46 | "not enable unless absolutely necessary! Experimental!"); 47 | DEFINE_int32(max_retained_connections_per_region_server, 2, 48 | "maximum number of connections to retain/cache per region server " 49 | "(note: actual number of connections may surpass this, up to " 50 | "--max_total_connections_per_region_server, to avoid blocking " 51 | "threads)"); 52 | DEFINE_int32(max_total_connections_per_region_server, 0, 53 | "maximum number of connections to cache per region server " 54 | "(hard limit, 0 for unlimited; will refuse connections if the " 55 | "total number of connections would surpass this amount)"); 56 | DEFINE_int32(hbase_max_row_buffer_size, 512, 57 | "maximum number of rows to buffer client side"); 58 | DEFINE_string(hbase_thrift_transport, "framed", 59 | "default transport to use when connecting to region servers; " 60 | "valid values: buffered header framed compact_framed"); 61 | DEFINE_string(hbase_thrift_compression, "none", 62 | "use compression for thrift communication; only valud if " 63 | "--hbase_thrift_transport is 'header'. Valid options: " 64 | "none zlib"); 65 | DEFINE_string(zookeeper_hbase_root_path, "/hbase/root-region-server", 66 | "zookeeper znode containing the root hbase path"); 67 | DEFINE_string(zookeeper_hbase_region_server_path, "/hbase/rs", 68 | "zookeeper znode containing region servers"); 69 | DEFINE_int32(zookeeper_reconnect_standoff_ms, 5000, "minimum time between " 70 | "attempts to connect to zookeeper"); 71 | DEFINE_bool(zookeeper_fast_fail, true, "if set, zookeeper reconnects are " 72 | "either instant or fail; if not set, retries will block"); 73 | DEFINE_int32(zookeeper_port, 2181, "port for communicating with zookeeper"); 74 | DEFINE_int32(hbase_default_timeout_ms, 60000, 75 | "timeout, in ms, for hbase RPCs"); 76 | 77 | namespace facebook { namespace hbase { 78 | 79 | TableNotFound::TableNotFound(const string& table, const string& row) 80 | : NativeHbaseException() { 81 | message_ = folly::stringPrintf("Table not found: %s (row %s)", 82 | table.c_str(), humanify(row).c_str()); 83 | } 84 | 85 | RegionServerConnectionError::RegionServerConnectionError( 86 | const string& host, int port) : NativeHbaseException() { 87 | message_ = folly::stringPrintf("Could not connect to region server: %s:%d", 88 | host.c_str(), port); 89 | } 90 | 91 | // This class is basically just a struct (for now). It contains the 92 | // start and end key for a given region, the host, and our internal 93 | // region number. It is immutable. 94 | class RegionServerInfo : private boost::noncopyable { 95 | public: 96 | RegionServerInfo(const TRegionInfo& r, int region_num) : 97 | start_row(r.startKey), end_row(r.endKey), host_valid(true), 98 | host(r.serverName), port(r.port), region_num(region_num) 99 | { } 100 | 101 | const string start_row; 102 | const string end_row; 103 | const bool host_valid; 104 | const string host; 105 | const int port; 106 | const int region_num; 107 | private: 108 | }; 109 | 110 | 111 | // A simple helper class for grouping a set of "things" by the region 112 | // server that serves a set of those "things". In this case, "things" 113 | // is any type that has a public "row" member (such as a 114 | // BatchMutation). Once constructed, this grouping can be iterated 115 | // over, resulting in one entry for all "things" served by the same 116 | // connection. 117 | // 118 | // For now, this is only used for RowMutation but it can be extended 119 | // for any kind of operation we wish to batch per region server. 120 | // Eventually it will be possible to dispatch operations in parallel 121 | // to different servers. 122 | template 123 | const string& extractKey(const T& t) { return t.row; } 124 | template<> 125 | const string& extractKey(const string& s) { return s; } 126 | 127 | template 128 | class ConnectionGroup : private boost::noncopyable { 129 | public: 130 | typedef std::unordered_map > Group; 131 | typedef typename Group::const_iterator Iter; 132 | 133 | ConnectionGroup(NativeHbaseClient* client, 134 | const string& table, 135 | const vector& entities) : client_(client) { 136 | group_.clear(); 137 | for (const T& e : entities) { 138 | auto conn = client->findRegion(table, extractKey(e), false); 139 | regions_known_.push_back(conn); 140 | group_[conn.get()].push_back(e); 141 | } 142 | } 143 | 144 | ~ConnectionGroup() { 145 | } 146 | 147 | Iter begin() { 148 | return group_.begin(); 149 | } 150 | 151 | Iter end() { 152 | return group_.end(); 153 | } 154 | 155 | private: 156 | NativeHbaseClient* client_; 157 | Group group_; 158 | vector > regions_known_; 159 | }; 160 | 161 | // Record RegionServerInfo 162 | shared_ptr 163 | TableRegions::addRegionServerInfo(const TRegionInfo& region, 164 | std::atomic* num_regions) { 165 | std::lock_guard lock(map_mutex_); 166 | auto ret = tableRegions_[region.startKey]; 167 | if (ret.get() == NULL) { 168 | // Not in the table, add it. 169 | ret.reset(new RegionServerInfo(region, ++*num_regions)); 170 | tableRegions_[region.startKey] = ret; 171 | } 172 | return ret; 173 | } 174 | 175 | // Find TableRegions in the map 176 | shared_ptr 177 | TableRegions::findRegionServerInfoInCache(const string& key) const { 178 | std::lock_guard lock(map_mutex_); 179 | // Maybe we got lucky and hit the exact row; return it. 180 | auto iter = tableRegions_.lower_bound(key); 181 | if (iter != tableRegions_.end()) { 182 | if (key >= iter->second->start_row && 183 | (key <= iter->second->end_row || iter->second->end_row.empty())) { 184 | return iter->second; 185 | } 186 | } 187 | 188 | // Otherwise the entry we found was the first one after the row we 189 | // sought; so, move the iterator back, unless we're already at the 190 | // first row, which means the row isn't in the table in question/ 191 | if (iter == tableRegions_.begin()) { 192 | return shared_ptr(); 193 | } 194 | --iter; 195 | if (key >= iter->second->start_row && 196 | (key <= iter->second->end_row || iter->second->end_row.empty())) { 197 | return iter->second; 198 | } 199 | return shared_ptr(); 200 | } 201 | 202 | void TableRegions::removeRegionServerInfo(const string& key) { 203 | std::lock_guard lock(map_mutex_); 204 | tableRegions_.erase(key); 205 | } 206 | 207 | // Given a string of the form host:port, split the host and port into 208 | // host_out and port_out. Returns true if the input string has the 209 | // right form. 210 | // 211 | // Magic processing is based on hbase 89's 212 | // org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper's 213 | // removeMetaData method. 214 | 215 | static bool ParseHostPort(const string& hostport, string* host_out, 216 | int* port_out) { 217 | // hbase 89 uses some magic "gunk" around the data it reads and 218 | // writes to zookeeper; the indicator of this gunk is the first byte 219 | // being the magic byte, 0xff. If we encounter this signifier, we 220 | // parse as per removeMetaData. If we do not, we treat it directly 221 | // as host:port (consistent with hbase 90). 222 | const char RECOVERABLE_ZOOKEEPER_MAGIC = 0xff; 223 | string real_hostport = hostport; 224 | if (real_hostport[0] == RECOVERABLE_ZOOKEEPER_MAGIC) { 225 | const int32_t MAGIC_OFFSET = sizeof(RECOVERABLE_ZOOKEEPER_MAGIC); 226 | const int32_t ID_OFFSET = sizeof(int32_t); 227 | const int32_t id_length = 228 | Endian::big32(*reinterpret_cast(hostport.data() + 229 | MAGIC_OFFSET)); 230 | const int32_t data_length = 231 | hostport.size() - MAGIC_OFFSET - ID_OFFSET - id_length; 232 | const int32_t data_offset = MAGIC_OFFSET + ID_OFFSET + id_length; 233 | real_hostport = string(hostport.data() + data_offset, data_length); 234 | } 235 | vector pieces; 236 | // Format: hbase 89 and 90 use a colon-separates 2-tuple of 237 | // host:port. hbase 92 uses a 3-tuple of host,port,timestamp. 238 | folly::split(",", real_hostport, pieces); 239 | if (pieces.size() == 1) { 240 | // No commas, that means hbase 89/90 that is just host:port. 241 | pieces.clear(); 242 | folly::split(":", real_hostport, pieces); 243 | } 244 | if (pieces.size() < 2) return false; 245 | 246 | *host_out = pieces[0]; 247 | *port_out = to(pieces[1]); 248 | return true; 249 | } 250 | 251 | const char* NativeHbaseClient::kWormholeHlogAttribute = "_walMeta"; 252 | 253 | static NativeHbaseConnection* 254 | make_connection(const NativeHbaseClient* client, 255 | const HostMapKey& key) { 256 | auto ret = new NativeHbaseConnection(key.first, key.second); 257 | if (!ret->connect(client->transport(), client->timeout())) { 258 | client->getCounters()->incrementCounter("connect_failures"); 259 | LOG(ERROR) << "Connect to region server failed: " 260 | << key.first << ":" << key.second; 261 | delete ret; 262 | return NULL; 263 | } 264 | client->getCounters()->incrementCounter("connect_successes"); 265 | return ret; 266 | } 267 | 268 | static HostMapKey 269 | key_from_connection(const NativeHbaseConnection* conn) { 270 | return make_pair(conn->host(), conn->port()); 271 | } 272 | 273 | NativeHbaseClient::NativeHbaseClient(const string& hosts, 274 | HbaseClientTransport transport) 275 | : healthy_(false), hosts_(hosts), 276 | zh_(NULL), transport_(transport), root_conn_(nullptr), num_regions_(0), 277 | rng_(time(NULL)), 278 | connection_pool_(FLAGS_max_retained_connections_per_region_server, 279 | FLAGS_max_total_connections_per_region_server, 280 | [=](const HostMapKey& key) { 281 | return make_connection(this, key); 282 | }, 283 | key_from_connection) { 284 | timeout_ms_ = FLAGS_hbase_default_timeout_ms; 285 | if (FLAGS_force_thrift_proxy_mode) { 286 | thrift_port_ = FLAGS_thrift_proxy_port; 287 | } else { 288 | thrift_port_ = FLAGS_hbase_port; 289 | } 290 | 291 | if (transport_ == DEFAULT_TRANSPORT) { 292 | if (FLAGS_hbase_thrift_transport == "buffered") { 293 | transport_ = BUFFERED_TRANSPORT; 294 | } else if (FLAGS_hbase_thrift_transport == "framed") { 295 | transport_ = FRAMED_TRANSPORT; 296 | } else if (FLAGS_hbase_thrift_transport == "compact_framed") { 297 | transport_ = COMPACT_FRAMED_TRANSPORT; 298 | } 299 | #if !NHC_OPEN_SOURCE 300 | else if (FLAGS_hbase_thrift_transport == "header") { 301 | transport_ = HEADER_TRANSPORT; 302 | if (FLAGS_hbase_thrift_compression == "zlib") { 303 | transport_ = HEADER_TRANSPORT_WITH_ZLIB; 304 | } 305 | } 306 | #endif 307 | else { 308 | LOG(FATAL) << "Invalid --hbase_thrift_transport: " 309 | << FLAGS_hbase_thrift_transport << ", must be " 310 | << "buffered or framed"; 311 | } 312 | } 313 | } 314 | 315 | void NativeHbaseClient::setCounter(shared_ptr counter) { 316 | stats_counters_ = counter; 317 | } 318 | 319 | static void empty_watcher(zhandle_t *zh, int type, int state, const char *path, 320 | void *context) {} 321 | 322 | 323 | bool 324 | NativeHbaseClient::expandConnectHosts(const string& specified_hosts, 325 | string* host_port_string, 326 | HostPortVector* host_ports) { 327 | vector pieces; 328 | folly::split(',', specified_hosts, pieces); 329 | for (const auto& host_port_str : pieces) { 330 | vector host_port_pieces; 331 | int port; 332 | folly::split(":", host_port_str, host_port_pieces); 333 | if (host_port_pieces.size() == 1) { 334 | port = FLAGS_zookeeper_port; 335 | } else if (host_port_pieces.size() == 2) { 336 | port = to(host_port_pieces[1]); 337 | } else { 338 | LOG(ERROR) << "host:port looks malformed: " << host_port_str; 339 | return false; 340 | } 341 | host_ports->push_back(make_pair(host_port_pieces[0], port)); 342 | } 343 | 344 | host_port_string->clear(); 345 | for (int i = 0; i < host_ports->size(); ++i) { 346 | if (i > 0) host_port_string->append(","); 347 | host_port_string->append( 348 | folly::stringPrintf( 349 | "%s:%d", (*host_ports)[i].first.c_str(), (*host_ports)[i].second)); 350 | } 351 | 352 | return true; 353 | } 354 | 355 | bool NativeHbaseClient::connectUnproteced() { 356 | if (!stats_counters_.get()) { 357 | stats_counters_.reset(new SimpleCounter()); 358 | } 359 | 360 | CHECK(!root_conn_mutex_.try_lock()) << "root connection mutex must be held!"; 361 | auto now = system_clock::now(); 362 | 363 | if (now - last_zookeeper_connect_time_ < 364 | milliseconds(FLAGS_zookeeper_reconnect_standoff_ms)) { 365 | if (FLAGS_zookeeper_fast_fail) { 366 | VLOG(1) << "Refusing to attempt reconnect to zookeeper, too recent"; 367 | return false; 368 | } else { 369 | VLOG(1) << "zk fastfail disabled, sleeping until allowed reconnect"; 370 | std::this_thread::sleep_until( 371 | milliseconds(FLAGS_zookeeper_reconnect_standoff_ms + 1) + 372 | last_zookeeper_connect_time_); 373 | now = system_clock::now(); 374 | } 375 | } 376 | last_zookeeper_connect_time_ = now; 377 | 378 | string hosts; 379 | if (!expandConnectHosts(hosts_, &hosts, &host_ports_)) { 380 | LOG(ERROR) << "Unable to resolve specified host string: " 381 | << hosts_; 382 | return false; 383 | } 384 | 385 | if (!FLAGS_force_thrift_proxy_mode) { 386 | zoo_set_debug_level(ZOO_LOG_LEVEL_WARN); 387 | zh_ = zookeeper_init(hosts.c_str(), empty_watcher, 10000, 0, 0, 0); 388 | if (!zh_) { 389 | LOG(ERROR) << "Unable to connect to zookeeper"; 390 | return false; 391 | } 392 | } 393 | 394 | return initRootConnection(); 395 | } 396 | 397 | bool NativeHbaseClient::connect() { 398 | std::lock_guard g1(root_conn_mutex_); 399 | return connectUnproteced(); 400 | } 401 | 402 | bool NativeHbaseClient::initRootConnection() { 403 | CHECK(!root_conn_mutex_.try_lock()) << "root connection mutex must be held!"; 404 | string host; 405 | int port; 406 | if (FLAGS_force_thrift_proxy_mode) { 407 | std::uniform_int_distribution picker(0, host_ports_.size()); 408 | const auto &chosen = host_ports_[picker(rng_)]; 409 | host = chosen.first; 410 | port = chosen.second; 411 | LOG(INFO) << "Choosing " << host << ":" << port << " as fake root region " 412 | << "server"; 413 | } else { 414 | if (!zh_) { 415 | if (!connectUnproteced()) { 416 | return false; 417 | } 418 | CHECK(zh_); 419 | } 420 | 421 | char contents[256]; 422 | int contents_length = sizeof(contents); 423 | int result = zoo_get(zh_, FLAGS_zookeeper_hbase_root_path.c_str(), 0, 424 | contents, &contents_length, NULL); 425 | if (result != ZOK) { 426 | LOG(ERROR) << "Error getting contents of " 427 | << FLAGS_zookeeper_hbase_root_path << ", result :" 428 | << zerror(result) << "; resetting zookeeper connection..."; 429 | zookeeper_close(zh_); 430 | zh_ = NULL; 431 | return false; 432 | } 433 | 434 | if (!ParseHostPort(string(contents, contents_length), &host, &port)) { 435 | LOG(ERROR) << "Unable to parse contents of " 436 | << FLAGS_zookeeper_hbase_root_path 437 | << ": " << string(contents, contents_length); 438 | return false; 439 | } 440 | port = thrift_port_; 441 | } 442 | 443 | root_conn_.reset(new NativeHbaseConnection(host, port)); 444 | 445 | if (root_conn_->connect(transport_, timeout_ms_)) { 446 | healthy_ = true; 447 | return true; 448 | } else { 449 | return false; 450 | } 451 | } 452 | 453 | NativeHbaseClient::~NativeHbaseClient() { 454 | if (zh_) { 455 | zookeeper_close(zh_); 456 | } 457 | 458 | if (VLOG_IS_ON(1)) { 459 | map stats; 460 | map histogram_stats; 461 | stats_counters_->snapshot(&stats, &histogram_stats); 462 | VLOG(1) << "Client stats dump:"; 463 | for (auto& it : stats) { 464 | VLOG(1) << " " << it.first << ": " << it.second; 465 | } 466 | VLOG(1) << "Client histogram dump:"; 467 | for (auto& it : histogram_stats) { 468 | VLOG(1) << " " << it.first << ": " << it.second; 469 | } 470 | } 471 | } 472 | 473 | bool 474 | NativeHbaseClient::listRegionServers(vector>* output) { 475 | output->clear(); 476 | if (FLAGS_force_thrift_proxy_mode) { 477 | *output = host_ports_; 478 | return true; 479 | } else { 480 | CHECK(zh_); 481 | struct String_vector sv; 482 | sv.count = 0; // all children 483 | int result = zoo_get_children( 484 | zh_, FLAGS_zookeeper_hbase_region_server_path.c_str(), 0, &sv); 485 | if (result != ZOK) { 486 | LOG(ERROR) << "Unable to retrieve list of region servers from zookeeper"; 487 | return false; 488 | } 489 | for (int i = 0; i < sv.count; ++i) { 490 | vector pieces; 491 | folly::split(',', sv.data[i], pieces); 492 | CHECK_GE(pieces.size(), 2) << "malformed region server entry: " 493 | << sv.data[i]; 494 | output->push_back(make_pair(pieces[0], 495 | to(pieces[1]))); 496 | } 497 | deallocate_String_vector(&sv); 498 | return true; 499 | } 500 | } 501 | 502 | void NativeHbaseClient::listTables(vector* output) { 503 | invokeRootOperation( 504 | this, "getTableNames", 505 | [=](HbaseClient* client) { 506 | client->getTableNames(*output); 507 | }); 508 | } 509 | 510 | void NativeHbaseClient::getColumnDescriptors( 511 | map *output, const string& table) { 512 | invokeRootOperation( 513 | this, "getColumnDescriptors", 514 | [=](HbaseClient* client) { 515 | return client->getColumnDescriptors(*output, table); 516 | }); 517 | } 518 | 519 | void NativeHbaseClient::createTable( 520 | const string& table, const vector& columns) { 521 | invokeRootOperation( 522 | this, "createTable", 523 | [=, &columns](HbaseClient* client) { 524 | return client->createTable(table, columns); 525 | }); 526 | } 527 | 528 | void NativeHbaseClient::deleteTable(const string& table) { 529 | invokeRootOperation( 530 | this, "deleteTable", 531 | [=](HbaseClient* client) { 532 | return client->deleteTable(table); 533 | }); 534 | } 535 | 536 | void NativeHbaseClient::enableTable(const string& table) { 537 | invokeRootOperation( 538 | this, "enableTable", 539 | [=](HbaseClient* client) { 540 | return client->enableTable(table); 541 | }); 542 | } 543 | 544 | void NativeHbaseClient::disableTable(const string& table) { 545 | invokeRootOperation( 546 | this, "disableTable", 547 | [=](HbaseClient* client) { 548 | return client->disableTable(table); 549 | }); 550 | } 551 | 552 | bool NativeHbaseClient::isTableEnabled(const string& table) { 553 | return invokeRootOperation( 554 | this, "isTableEnabled", 555 | [=](HbaseClient* client) { 556 | return client->isTableEnabled(table); 557 | }); 558 | } 559 | 560 | void NativeHbaseClient::listTableRegions(vector* output, 561 | const string& table) { 562 | invokeRootOperation( 563 | this, "getTableRegions", 564 | [=](HbaseClient* client) { 565 | return client->getTableRegions(*output, table); 566 | }); 567 | } 568 | 569 | void NativeHbaseClient::getRow(vector* output, 570 | const string& table, const string& row) { 571 | stats_counters_->incrementCounter("queries"); 572 | invokeRowOperation( 573 | table, row, "getRow", 574 | [=](HbaseClient* client) { 575 | return client->getRow(*output, table, row); 576 | }); 577 | } 578 | 579 | void NativeHbaseClient::getRowWithColumns( 580 | vector* output, 581 | const string& table, 582 | const string& row, 583 | const vector& column_specifiers) { 584 | stats_counters_->incrementCounter("queries"); 585 | invokeRowOperation( 586 | table, row, "getRowWithColumns", 587 | [=, &column_specifiers](HbaseClient* client) { 588 | return client->getRowWithColumns(*output, table, row, column_specifiers); 589 | }); 590 | } 591 | 592 | void NativeHbaseClient::getRowWithColumnsTs( 593 | vector* output, 594 | const string& table, 595 | const string& row, 596 | const vector& column_specifiers, 597 | const int64_t timestamp) { 598 | stats_counters_->incrementCounter("queries"); 599 | invokeRowOperation( 600 | table, row, "getRowWithColumnsTs", 601 | [=, &column_specifiers](HbaseClient* client) { 602 | return client->getRowWithColumnsTs(*output, table, row, 603 | column_specifiers, timestamp); 604 | }); 605 | } 606 | 607 | bool NativeHbaseClient::getRowsHelper( 608 | vector* output, 609 | unordered_map* output_by_key, 610 | const string& table, 611 | const vector& keys, 612 | const char* opname, 613 | GetRowsHelperOp op) 614 | { 615 | stats_counters_->incrementCounter("queries"); 616 | ConnectionGroup mutations(this, table, keys); 617 | 618 | output->clear(); 619 | // We want pointers into output to be stable across all passes, so 620 | // we pre-allocate the entries. TODO(chip): make this API cleaner; 621 | // andrei suggests possibly using a boost::multi_index. Other 622 | // options are making the api cleaner by having two calls (one with 623 | // a vector, one with a map) or making copies of values for the row 624 | // output_by_key results. 625 | output->reserve(keys.size()); 626 | bool okay = true; 627 | for (ConnectionGroup::Iter it = mutations.begin(); 628 | it != mutations.end(); ++it) { 629 | bool failed_query = true; 630 | const vector& keys = it->second; 631 | try { 632 | vector batch_results; 633 | invokeRowOperation( 634 | table, it->first->start_row, opname, 635 | [&op, &batch_results, &table, &keys](HbaseClient* client) { 636 | return op(client, batch_results, table, keys); 637 | }); 638 | 639 | 640 | for (auto& result : batch_results) { 641 | output->push_back(result); 642 | if (output_by_key) { 643 | // output->back() is stable across inserts into output 644 | // because of the earlier resize. 645 | (*output_by_key)[result.row] = &output->back(); 646 | } 647 | } 648 | 649 | if (output_by_key) { 650 | for (auto& key : keys) { 651 | // insert doesn't overwrite, so fill in all keys not present 652 | // in our result set with nullptr's. 653 | output_by_key->insert(make_pair(key, nullptr)); 654 | } 655 | } 656 | 657 | failed_query = false; 658 | } 659 | catch (const IllegalArgument& e) { 660 | LOG(ERROR) << opname << " failed: " << e.message; 661 | } 662 | catch (const IOError& e) { 663 | LOG(ERROR) << opname << " failed: " << e.message; 664 | } 665 | catch (const std::exception& e) { 666 | LOG(ERROR) << opname << " failed: " << e.what(); 667 | } 668 | 669 | if (failed_query && output_by_key) { 670 | for (auto& key : keys) { 671 | output_by_key->erase(key); 672 | } 673 | } 674 | 675 | okay &= !failed_query; 676 | } 677 | 678 | return okay; 679 | } 680 | 681 | bool NativeHbaseClient::getRows( 682 | vector* output, 683 | unordered_map* output_by_key, 684 | const string& table, 685 | const vector& keys) 686 | { 687 | GetRowsHelperOp op = 688 | [](HbaseClient* client, 689 | vector& batch_results, 690 | const string& table, 691 | const vector& keys) { 692 | return client->getRows(batch_results, table, keys); 693 | }; 694 | 695 | return getRowsHelper(output, output_by_key, table, keys, "getRows", op); 696 | } 697 | 698 | bool NativeHbaseClient::getRowsWithColumns( 699 | vector* output, 700 | unordered_map* output_by_key, 701 | const string& table, 702 | const vector& keys, 703 | const vector& columns) 704 | { 705 | GetRowsHelperOp op = 706 | [&columns](HbaseClient* client, 707 | vector& batch_results, 708 | const string& table, 709 | const vector& keys) { 710 | return client->getRowsWithColumns(batch_results, table, 711 | keys, columns); 712 | }; 713 | 714 | return getRowsHelper(output, output_by_key, table, keys, 715 | "getRowsWithColumns", op); 716 | } 717 | 718 | Scanner* NativeHbaseClient::getScanner(const string& table, 719 | const string& start_row, 720 | const vector& columns) { 721 | auto region_info = findRegion(table, start_row, false); 722 | return new Scanner(this, table, start_row, 723 | region_info->start_row, region_info->end_row, columns); 724 | } 725 | 726 | bool NativeHbaseClient::mutateRow(const string& table, const string& row, 727 | const vector& mutations) { 728 | const string empty_string; 729 | return mutateRow(table, row, mutations, empty_string); 730 | } 731 | 732 | bool NativeHbaseClient::mutateRow(const string& table, const string& row, 733 | const vector& mutations, 734 | const string& transaction_annotation) { 735 | NativeHbaseConnection* conn = findConnection(table, row, false); 736 | ScopeGuard g = makeGuard([=] { connection_pool_.release(conn); } ); 737 | 738 | try { 739 | map attributes; 740 | if (!transaction_annotation.empty()) { 741 | attributes[kWormholeHlogAttribute] = transaction_annotation; 742 | } 743 | invokeRowOperation( 744 | table, row, "mutateRow", 745 | [=, &mutations, &attributes](HbaseClient* client) { 746 | return client->mutateRow(table, row, mutations, attributes); 747 | }); 748 | return true; 749 | } 750 | catch (const IOError& e) { 751 | LOG(ERROR) << "mutateRow failed: " << e.message; 752 | } 753 | catch (const IllegalArgument& e) { 754 | LOG(ERROR) << "mutateRow failed: " << e.message; 755 | } 756 | catch (const std::exception& e) { 757 | LOG(ERROR) << "mutateRow failed: " << e.what(); 758 | } 759 | return false; 760 | } 761 | 762 | bool NativeHbaseClient::mutateRowTs(const string& table, const string& row, 763 | const vector& mutations, 764 | const int64_t timestamp) { 765 | const string empty_string; 766 | return mutateRowTs(table, row, mutations, timestamp, empty_string); 767 | } 768 | 769 | bool NativeHbaseClient::mutateRowTs(const string& table, const string& row, 770 | const vector& mutations, 771 | const int64_t timestamp, 772 | const string& transaction_annotation) { 773 | try { 774 | map attributes; 775 | if (!transaction_annotation.empty()) { 776 | attributes[kWormholeHlogAttribute] = transaction_annotation; 777 | } 778 | invokeRowOperation( 779 | table, row, "mutateRowTs", 780 | [=, &mutations, &attributes](HbaseClient* client) { 781 | return client->mutateRowTs(table, row, mutations, 782 | timestamp, attributes); 783 | }); 784 | 785 | return true; 786 | } 787 | catch (const IOError& e) { 788 | LOG(ERROR) << "mutateRowTs failed: " << e.message; 789 | } 790 | catch (const IllegalArgument& e) { 791 | LOG(ERROR) << "mutateRowTs failed: " << e.message; 792 | } 793 | catch (const std::exception& e) { 794 | LOG(ERROR) << "mutateRowTs failed: " << e.what(); 795 | } 796 | return false; 797 | } 798 | 799 | bool NativeHbaseClient::mutateRows(const string& table, 800 | const vector& rowBatches) { 801 | const string empty_string; 802 | return mutateRows(table, rowBatches, empty_string); 803 | } 804 | 805 | bool NativeHbaseClient::mutateRows(const string& table, 806 | const vector& rowBatches, 807 | const string& transaction_annotation) { 808 | ConnectionGroup mutations(this, table, rowBatches); 809 | 810 | bool okay = true; 811 | for (ConnectionGroup::Iter it = mutations.begin(); 812 | it != mutations.end(); ++it) { 813 | try { 814 | map attributes; 815 | if (!transaction_annotation.empty()) { 816 | attributes[kWormholeHlogAttribute] = transaction_annotation; 817 | } 818 | invokeRowOperation( 819 | table, it->first->start_row, "mutateRows", 820 | [=, &it, &attributes](HbaseClient* client) { 821 | return client->mutateRows(table, it->second, attributes); 822 | }); 823 | } 824 | catch (const IllegalArgument& e) { 825 | LOG(ERROR) << "mutateRows failed: " << e.message; 826 | okay = false; 827 | } 828 | catch (const IOError& e) { 829 | LOG(ERROR) << "mutateRows failed: " << e.message; 830 | okay = false; 831 | } 832 | catch (const std::exception& e) { 833 | LOG(ERROR) << "mutateRows failed: " << e.what(); 834 | okay = false; 835 | } 836 | } 837 | 838 | return okay; 839 | } 840 | 841 | bool NativeHbaseClient::mutateRowsTs(const string& table, 842 | const vector& rowBatches, 843 | const int64_t timestamp) { 844 | const string empty_string; 845 | return mutateRowsTs(table, rowBatches, timestamp, empty_string); 846 | } 847 | 848 | bool NativeHbaseClient::mutateRowsTs(const string& table, 849 | const vector& rowBatches, 850 | const int64_t timestamp, 851 | const string& transaction_annotation) { 852 | ConnectionGroup mutations(this, table, rowBatches); 853 | 854 | bool okay = true; 855 | for (ConnectionGroup::Iter it = mutations.begin(); 856 | it != mutations.end(); ++it) { 857 | try { 858 | map attributes; 859 | if (!transaction_annotation.empty()) { 860 | attributes[kWormholeHlogAttribute] = transaction_annotation; 861 | } 862 | invokeRowOperation( 863 | table, it->first->start_row, "mutateRowsTs", 864 | [=, &it, &attributes](HbaseClient* client) { 865 | return client->mutateRowsTs(table, it->second, 866 | timestamp, attributes); 867 | }); 868 | } 869 | catch (const IllegalArgument& e) { 870 | LOG(ERROR) << "mutateRowsTs failed: " << e.message; 871 | okay = false; 872 | } 873 | catch (const IOError& e) { 874 | LOG(ERROR) << "mutateRowsTs failed: " << e.message; 875 | okay = false; 876 | } 877 | catch (const std::exception& e) { 878 | LOG(ERROR) << "mutateRowsTs failed: " << e.what(); 879 | okay = false; 880 | } 881 | } 882 | 883 | return okay; 884 | } 885 | 886 | bool NativeHbaseClient::deleteAll(const string& table, const string& row, 887 | const string& column) { 888 | try { 889 | invokeRowOperation( 890 | table, row, "deleteAll", 891 | [=](HbaseClient* client) { 892 | client->deleteAll(table, row, column); 893 | }); 894 | 895 | return true; 896 | } 897 | catch (const IOError& e) { 898 | LOG(ERROR) << "deleteAll failed: " << e.message; 899 | } 900 | catch (const std::exception& e) { 901 | LOG(ERROR) << "deleteAll failed: " << e.what(); 902 | } 903 | return false; 904 | } 905 | 906 | bool NativeHbaseClient::deleteAllTs(const string& table, const string& row, 907 | const string& column, 908 | const int64_t timestamp) { 909 | try { 910 | invokeRowOperation( 911 | table, row, "deleteAllTs", 912 | [=](HbaseClient* client) { 913 | client->deleteAllTs(table, row, column, timestamp); 914 | }); 915 | 916 | return true; 917 | } 918 | catch (const IOError& e) { 919 | LOG(ERROR) << "deleteAllTs failed: " << e.message; 920 | } 921 | catch (const std::exception& e) { 922 | LOG(ERROR) << "deleteAllTs failed: " << e.what(); 923 | } 924 | return false; 925 | } 926 | 927 | bool NativeHbaseClient::deleteAllRow(const string& table, const string& row) { 928 | const string empty_string; 929 | return deleteAllRow(table, row, empty_string); 930 | } 931 | 932 | bool NativeHbaseClient::deleteAllRow(const string& table, const string& row, 933 | const string& transaction_annotation) { 934 | try { 935 | map attributes; 936 | if (!transaction_annotation.empty()) { 937 | attributes[kWormholeHlogAttribute] = transaction_annotation; 938 | } 939 | invokeRowOperation( 940 | table, row, "deleteAllRow", 941 | [=, &attributes](HbaseClient* client) { 942 | client->deleteAllRow(table, row, attributes); 943 | }); 944 | 945 | return true; 946 | } 947 | catch (const IOError& e) { 948 | LOG(ERROR) << "deleteAllRow failed: " << e.message; 949 | } 950 | catch (const std::exception& e) { 951 | LOG(ERROR) << "deleteAllRow failed: " << e.what(); 952 | } 953 | return false; 954 | } 955 | 956 | bool NativeHbaseClient::deleteAllRowTs(const string& table, const string& row, 957 | const int64_t timestamp) { 958 | try { 959 | invokeRowOperation( 960 | table, row, "deleteAllRowTs", 961 | [=](HbaseClient* client) { 962 | client->deleteAllRowTs(table, row, timestamp); 963 | }); 964 | 965 | return true; 966 | } 967 | catch (const IOError& e) { 968 | LOG(ERROR) << "deleteAllRowTs failed: " << e.message; 969 | } 970 | catch (const std::exception& e) { 971 | LOG(ERROR) << "deleteAllRowTs failed: " << e.what(); 972 | } 973 | return false; 974 | } 975 | 976 | int64_t NativeHbaseClient::atomicIncrement(const string& table, 977 | const string& row, 978 | const string& column, 979 | int64_t delta) { 980 | return invokeRowOperation( 981 | table, row, "atomicIncrement", 982 | [=](HbaseClient* client) { 983 | return client->atomicIncrement(table, row, column, delta); 984 | }); 985 | } 986 | 987 | Scanner::Scanner(NativeHbaseClient* client, const string& table, 988 | const string& start_row, const string®ion_start_row, 989 | const string& region_end_row, const vector& columns) 990 | : state_(READY), client_(client), table_(table), columns_(columns), 991 | start_row_(start_row), region_start_row_(region_start_row), 992 | region_end_row_(region_end_row), 993 | region_scanner_(make_pair(static_cast(NULL), -1)), 994 | row_buffer_capacity_(16), row_cursor_pos_(-1) { 995 | } 996 | 997 | Scanner::~Scanner() { 998 | if (region_scanner_.first) { 999 | client_->releaseConnection(region_scanner_.first); 1000 | } 1001 | } 1002 | 1003 | void Scanner::setEndRow(const string& end_row) { 1004 | DCHECK_EQ(state_, READY); 1005 | // The HBase server doesn't treat this as inclusive, so append a 1006 | // null byte; there are no rows lexigraphically between end_row_ and 1007 | // end_row_ + "\0" so this gets the result we want. 1008 | end_row_ = end_row + string("", 1); 1009 | } 1010 | 1011 | void Scanner::setRowBufferCapacity(const int n) { 1012 | DCHECK_EQ(state_, READY); 1013 | row_buffer_capacity_ = n; 1014 | } 1015 | 1016 | void Scanner::scan() { 1017 | DCHECK_EQ(state_, READY); 1018 | if (state_ != READY) { 1019 | state_ = ERROR; 1020 | return; 1021 | } 1022 | 1023 | // There are a few cases to consider with respect to the current 1024 | // region's boundaries and end_row_ (if it is specified). 1025 | // 1026 | // If end_row_ isn't specified, scan to the end of this region. 1027 | // 1028 | // If end_row_ is specified, and it falls within the boundaries of 1029 | // the current region, use it instead of the region end key. 1030 | // 1031 | 1032 | string this_scan_end_row; 1033 | if (region_end_row_.size() > 0) { 1034 | this_scan_end_row = region_end_row_ + string("", 1); 1035 | } 1036 | 1037 | if (end_row_.size() > 0) { 1038 | if (start_row_ <= end_row_ && 1039 | (region_end_row_.empty() || end_row_ <= region_end_row_)) { 1040 | this_scan_end_row = end_row_; 1041 | } 1042 | } 1043 | 1044 | region_scanner_ = client_->beginScanInternal(table_, columns_, 1045 | start_row_, this_scan_end_row); 1046 | state_ = SCANNING; 1047 | } 1048 | 1049 | void Scanner::advanceRegion() { 1050 | client_->releaseConnection(region_scanner_.first); 1051 | 1052 | // If any code before the final assignment in this function throws 1053 | // an exception, including the beginScanInternal call in the 1054 | // assignment statement, we still have a raw pointer inside the 1055 | // region_scanner_ pair that our destructor will release. That 1056 | // would be a double release! Aiee. 1057 | region_scanner_.first = NULL; 1058 | 1059 | // We begin the next row at the region containing the next key after 1060 | // the end of this region. Use the lexigraphic null byte trick again. 1061 | const string next_row = region_end_row_ + string("", 1); 1062 | auto region_info = 1063 | client_->findRegion(table_, next_row, false); 1064 | region_start_row_ = region_info->start_row; 1065 | region_end_row_ = region_info->end_row; 1066 | 1067 | string this_scan_end_row = region_end_row_; 1068 | if (this_scan_end_row.size() > 0) { 1069 | this_scan_end_row += string("", 1); 1070 | if (end_row_.size() > 0) { 1071 | this_scan_end_row = min(end_row_, this_scan_end_row); 1072 | } 1073 | } 1074 | 1075 | region_scanner_ = client_->beginScanInternal(table_, columns_, 1076 | next_row, this_scan_end_row); 1077 | } 1078 | 1079 | bool Scanner::next(TRowResult** output) { 1080 | if (state_ != SCANNING) { 1081 | return false; 1082 | } 1083 | // Has our cursor run past the end of the buffer? If so, refill. 1084 | if (row_cursor_pos_ >= row_buffer_.size()) { 1085 | if (row_buffer_.size() < FLAGS_hbase_max_row_buffer_size) { 1086 | row_buffer_capacity_ = std::min(2 * row_buffer_capacity_, 1087 | FLAGS_hbase_max_row_buffer_size); 1088 | } 1089 | 1090 | row_buffer_.clear(); 1091 | row_buffer_.reserve(row_buffer_capacity_); 1092 | row_cursor_pos_ = 0; 1093 | 1094 | // Try to get rows from successive regions until we get rows or hit an error 1095 | while (true) { 1096 | if (!client_->getNextRows(&row_buffer_, region_scanner_, 1097 | row_buffer_capacity_)) { 1098 | // we hit an error, terminate early 1099 | state_ = ERROR; 1100 | *output = NULL; 1101 | return false; 1102 | } else if (row_buffer_.empty()) { 1103 | // No more rows in this region; either we are done, or we need 1104 | // to advance to the next region. 1105 | if (region_end_row_.empty() || 1106 | (!end_row_.empty() && region_start_row_ > end_row_)) { 1107 | // No more regions! We are done. 1108 | *output = NULL; 1109 | state_ = FINISHED; 1110 | return false; 1111 | } 1112 | advanceRegion(); 1113 | } else { 1114 | // The buffer has been refilled, carry on 1115 | break; 1116 | } 1117 | } 1118 | } 1119 | 1120 | *output = &(row_buffer_[row_cursor_pos_++]); 1121 | if (!end_row_.empty() && (*output)->row > end_row_) { 1122 | // Are we past our stop row? Then we are done. 1123 | *output = NULL; 1124 | state_ = FINISHED; 1125 | return false; 1126 | } 1127 | return true; 1128 | } 1129 | 1130 | RegionScanner NativeHbaseClient::beginScanInternal( 1131 | const string& table, const vector& columns, 1132 | const string& start_row, const string& end_row) { 1133 | NativeHbaseConnection* conn = findConnection(table, start_row, false); 1134 | ScopeGuard g = makeGuard([=] { connection_pool_.release(conn); } ); 1135 | 1136 | ScannerID scanner_id = invokeThriftOperation( 1137 | this, 1138 | conn, 1139 | [=, &columns](HbaseClient* client) { 1140 | return client->scannerOpenWithStop(table, start_row, end_row, columns); 1141 | }); 1142 | 1143 | g.dismiss(); 1144 | return make_pair(conn, scanner_id); 1145 | } 1146 | 1147 | bool NativeHbaseClient::getNextRows(vector* output, 1148 | RegionScanner region_scanner, 1149 | int num_rows) { 1150 | try { 1151 | invokeThriftOperation( 1152 | this, 1153 | region_scanner.first, 1154 | [=, ®ion_scanner](HbaseClient* client) { 1155 | client->scannerGetList(*output, region_scanner.second, num_rows); 1156 | }); 1157 | return true; 1158 | } 1159 | catch (const IOError& e) { 1160 | LOG(ERROR) << "scannerGetlist failed: " << e.message; 1161 | } 1162 | catch (const IllegalArgument& e) { 1163 | LOG(ERROR) << "scannerGetList failed: " << e.message; 1164 | } 1165 | catch (const std::exception& e) { 1166 | LOG(ERROR) << "scannerGetList failed: " << e.what(); 1167 | } 1168 | return false; 1169 | } 1170 | 1171 | NativeHbaseConnection* NativeHbaseClient::connectionForRegionServer( 1172 | const RegionServerInfo* region_server_info) { 1173 | HostMapKey key(region_server_info->host, thrift_port_); 1174 | auto ret = connection_pool_.lookup(key); 1175 | if (ret == NULL) { 1176 | throw RegionServerConnectionError(region_server_info->host, 1177 | thrift_port_); 1178 | } 1179 | return ret; 1180 | } 1181 | 1182 | shared_ptr 1183 | NativeHbaseClient::findRegion(const string& table, 1184 | const string& row, 1185 | bool ignore_cache) { 1186 | regions_mutex_.lock(); 1187 | TableMap::const_iterator it = regions_.find(table); 1188 | TableRegions* table_entry = NULL; 1189 | if (it == regions_.end()) { 1190 | regions_[table].reset(new TableRegions); 1191 | table_entry = regions_[table].get(); 1192 | } else { 1193 | table_entry = it->second.get(); 1194 | } 1195 | regions_mutex_.unlock(); 1196 | assert(table_entry); 1197 | 1198 | shared_ptr ret = 1199 | table_entry->findRegionServerInfoInCache(row); 1200 | 1201 | if (ret.get() && !ignore_cache) { 1202 | VLOG(1) << "Cache hit finding region server for " << table 1203 | << " row " << humanify(row) << " (region range is " 1204 | << humanify(ret->start_row) << "-" 1205 | << humanify(ret->end_row) << ")"; 1206 | return ret; 1207 | } 1208 | 1209 | VLOG(1) << "Finding region server for " << table << " row " 1210 | << humanify(row); 1211 | 1212 | TRegionInfo region; 1213 | static const string kNines = "99999999999999"; // from HConstats.java 1214 | string meta_row_key = table + "," + row + "," + kNines; 1215 | 1216 | try { 1217 | VLOG(1) << "getRegionInfo " << humanify(meta_row_key); 1218 | // TODO(chip): this should ask the region server containing the 1219 | // correct region of META rather than the root client. 1220 | invokeRootOperation( 1221 | this, "getRegionInfo", 1222 | [=, ®ion](HbaseClient* client) { 1223 | client->getRegionInfo(region, meta_row_key); 1224 | }); 1225 | } 1226 | catch (const IOError& e) { 1227 | LOG(ERROR) << "getRegionInfo failed: " << e.message; 1228 | throw TableNotFound(table, row); 1229 | } 1230 | 1231 | assert(row >= region.startKey); 1232 | 1233 | if (row <= region.endKey || region.endKey.empty()) { 1234 | if (ret.get()) { 1235 | table_entry->removeRegionServerInfo(ret->start_row); 1236 | } 1237 | ret = table_entry->addRegionServerInfo(region, &num_regions_); 1238 | } 1239 | 1240 | if (!ret) { 1241 | throw TableNotFound(table, row); 1242 | } 1243 | 1244 | return ret; 1245 | } 1246 | 1247 | }} // namespace 1248 | 1249 | std::ostream& operator<<(std::ostream& out, 1250 | const std::pair& key) { 1251 | out << key.first << ":" << key.second; 1252 | return out; 1253 | } 1254 | 1255 | -------------------------------------------------------------------------------- /hbase/NativeHbaseClient.h: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // @author Chip Turner (chip@fb.com) 17 | // 18 | // This is the "Native" HBase client; specifically, it does not 19 | // require the use of the Java Thrift proxy (aka ThriftServer); 20 | // instead, it communicates directly with the servers containing data. 21 | // 22 | // See hbc.cpp for example usage. 23 | // 24 | // This library is designed to be thread safe. A single connection 25 | // can be instantiated against a given hbase cluster, and this 26 | // connection can safely be used across threads without external 27 | // locking. Scanners should not be shared across threads; if a thread 28 | // begins a scan, the scanner operation should be completed on that 29 | // thread. Other threads can acquire their own scanners. Multiple 30 | // scanners can be used in parallel from the same connection. 31 | // 32 | // DANGER: This client is a work in progress. If something breaks, 33 | // you get to keep both pieces. 34 | // 35 | // On the use of Mutexes in the implementation of this class: Mutexes 36 | // are rarely held concurrently, but then they are, the order should 37 | // be: 38 | // 39 | // root_conn_mutex_ -> regions_mutex_ 40 | 41 | #ifndef HBASE_SRC_NATIVEHBASECLIENT_H_ 42 | #define HBASE_SRC_NATIVEHBASECLIENT_H_ 43 | 44 | #include "hbase/nhc-config.h" 45 | 46 | #if NHC_OPEN_SOURCE 47 | #include 48 | #else 49 | #include 50 | #endif 51 | 52 | #include 53 | #include 54 | #include 55 | #include 56 | #include 57 | #include 58 | #include 59 | 60 | #include 61 | #include 62 | 63 | #include "folly/Conv.h" 64 | #include "folly/ScopeGuard.h" 65 | #include "hbase/Hbase.h" 66 | #include "hbase/BaseHbaseConnection.h" 67 | #include "hbase/ConnectionPool.h" 68 | #include "hbase/Counters.h" 69 | #include "concurrency/Mutex.h" 70 | #include "Thrift.h" 71 | 72 | namespace facebook { namespace hbase { 73 | 74 | using boost::shared_ptr; 75 | 76 | using apache::hadoop::hbase::thrift::AlreadyExists; 77 | using apache::hadoop::hbase::thrift::BatchMutation; 78 | using apache::hadoop::hbase::thrift::ColumnDescriptor; 79 | using apache::hadoop::hbase::thrift::HbaseClient; 80 | using apache::hadoop::hbase::thrift::IOError; 81 | using apache::hadoop::hbase::thrift::IllegalArgument; 82 | using apache::hadoop::hbase::thrift::Mutation; 83 | using apache::hadoop::hbase::thrift::ScannerID; 84 | using apache::hadoop::hbase::thrift::TCell; 85 | using apache::hadoop::hbase::thrift::TRegionInfo; 86 | using apache::hadoop::hbase::thrift::TRowResult; 87 | using apache::hadoop::hbase::thrift::Text; 88 | using apache::thrift::concurrency::Mutex; 89 | using apache::thrift::transport::TTransportException; 90 | #if !NHC_OPEN_SOURCE 91 | using apache::thrift::TLibraryException; 92 | #endif 93 | using folly::humanify; 94 | using folly::ScopeGuard; 95 | using folly::to; 96 | using std::pair; 97 | using std::string; 98 | using std::vector; 99 | using std::unordered_map; 100 | using std::unique_ptr; 101 | using std::chrono::microseconds; 102 | using std::chrono::milliseconds; 103 | using std::chrono::system_clock; 104 | 105 | class FacebookHbaseClient; 106 | class NativeHbaseClient; 107 | class RegionServerInfo; 108 | 109 | #if NHC_OPEN_SOURCE 110 | typedef TException NativeHbaseException; 111 | #else 112 | typedef TLibraryException NativeHbaseException; 113 | #endif 114 | 115 | // Exceptions beyond that which may be thrown by the thrift layers. 116 | // TODO(chip): perhaps we should have a base class for all exceptions 117 | // the client may throw, and wrap thrift exceptions in them? Unclear 118 | // at this time. 119 | class TableNotFound : public NativeHbaseException { 120 | public: 121 | TableNotFound(const string& table, const string& row); 122 | }; 123 | 124 | class RegionServerConnectionError : public NativeHbaseException { 125 | public: 126 | RegionServerConnectionError(const string& host, int port); 127 | }; 128 | 129 | // map: first_row -> regionserver with lock protection 130 | class TableRegions : private boost::noncopyable { 131 | public: 132 | /** 133 | * Add the region to map. If the region exist in the map, just return the 134 | * exist data 135 | * @param region for creating the RegionServerInfo 136 | * @param num_regions current num of region in current TableMap 137 | * @returns RegionServerInfo either created or in the map 138 | */ 139 | shared_ptr addRegionServerInfo( 140 | const TRegionInfo& region, 141 | std::atomic* num_regions); 142 | 143 | /** 144 | * Find this key in the map 145 | * @param key key to find the RegionServerInfo 146 | * @returns RegionServerInfo, otherwise NULL if not found 147 | */ 148 | shared_ptr findRegionServerInfoInCache( 149 | const string& key) const; 150 | 151 | /** 152 | * Remove the region server info for the specified start key. 153 | */ 154 | void removeRegionServerInfo(const string& start_key); 155 | 156 | private: 157 | mutable std::mutex map_mutex_; // protect local map 158 | map > tableRegions_; 159 | }; 160 | 161 | // a vector of host:port's 162 | typedef vector> HostPortVector; 163 | 164 | // map: table name -> TableRegions 165 | typedef unordered_map> TableMap; 166 | 167 | // pair of host, port 168 | typedef pair HostMapKey; 169 | 170 | // a handy type for our connections for most hbase RPCs 171 | typedef BaseHbaseConnection NativeHbaseConnection; 172 | 173 | // an active region scan operation 174 | typedef pair RegionScanner; 175 | 176 | // our connection pool type (for brevity later) 177 | typedef ConnectionPool HbaseConnectionPool; 178 | 179 | template 180 | class ThriftOperation { 181 | public: 182 | typedef std::function Type; 183 | }; 184 | 185 | /** 186 | * class Scanner 187 | * 188 | * Represents a scan operation. Returned by NativeHbaseClient and 189 | * used to move forward and access rows returned from the scan. 190 | */ 191 | class Scanner : private boost::noncopyable { 192 | public: 193 | // No public constructor. 194 | ~Scanner(); 195 | 196 | /// Set the last row to be returned; scan stops no later than this row. 197 | void setEndRow(const string& end_row); 198 | /// Set the number of rows to buffer; trade throughput for latency. 199 | void setRowBufferCapacity(const int n); 200 | 201 | /// Begin the scan itself. 202 | void scan(); 203 | 204 | /** 205 | * Advance to the next row, returning false if there are no more rows. 206 | * 207 | * @param output set to the current row 208 | */ 209 | bool next(TRowResult** output); 210 | 211 | /// Returns true iff the scanner encountered an error fetching rows. 212 | bool error() const { return state_ == ERROR; } 213 | 214 | private: 215 | enum State { 216 | READY, 217 | SCANNING, 218 | ERROR, 219 | FINISHED 220 | }; 221 | State state_; 222 | NativeHbaseClient* client_; // Client used by this scanner. 223 | const string table_; // table being scanned 224 | const vector columns_; // columns being returned 225 | const string start_row_; // where the scan starts 226 | string end_row_; // upper bound on rows returned 227 | string region_start_row_; // first row in this current region 228 | string region_end_row_; // last row in this current region 229 | 230 | RegionScanner region_scanner_; 231 | 232 | // We fetch multiple rows at once, but the API results in a single 233 | // row; we buffer them here. 234 | vector row_buffer_; 235 | 236 | int row_buffer_capacity_; // buffer capacity 237 | int row_cursor_pos_; // current position in the row buffer 238 | 239 | // Move scanner to next region. 240 | void advanceRegion(); 241 | 242 | /** 243 | * Private constructor used by NativeHbaseClient. 244 | * 245 | * @param client the client creating this scanner 246 | * @param start_row the row to begin the scan at 247 | * @param columns vector of columns to fetch 248 | */ 249 | Scanner(NativeHbaseClient* client, 250 | const string& table, 251 | const string& start_row, 252 | const string& region_start_row, 253 | const string& region_end_row, 254 | const vector& columns); 255 | 256 | friend class NativeHbaseClient; 257 | }; 258 | 259 | 260 | /** 261 | * class NativeHbaseClient 262 | * 263 | * The C++ client initiates communication with any region server; from 264 | * there, it locates the relevant metadata to find regions for 265 | * particular rows. The host:port specified in the constructor is 266 | * effectively just a boostrapping region server. 267 | */ 268 | class NativeHbaseClient : private boost::noncopyable { 269 | public: 270 | /** 271 | * Constructor. 272 | * 273 | * @param smc_tier_or_hosts either an smc tier or a comma separated 274 | * list of host:port for our zookeeper instance 275 | * @param transport a HbaseClientTransport indicating the transport for Thrift 276 | */ 277 | explicit NativeHbaseClient( 278 | const string& smc_tier_or_hosts, 279 | HbaseClientTransport transport = DEFAULT_TRANSPORT); 280 | virtual ~NativeHbaseClient(); 281 | 282 | /** 283 | * A wrapper that invoked a ThriftOperation for a given table/row. 284 | * This will retry once if an error is encountered, display a 285 | * message, etc. 286 | */ 287 | template 288 | T invokeRowOperation(const string& table, const string& row, 289 | const char* opname, 290 | typename ThriftOperation::Type op); 291 | 292 | /** 293 | * Similar to the above, but this operation is invoked on the root client. 294 | */ 295 | template 296 | T invokeRootOperation(NativeHbaseClient* client, const char* opname, 297 | typename ThriftOperation::Type op); 298 | 299 | /** 300 | * Log an error message. Intended to be overridden by subclasses 301 | * and used for custom logging purposes. 302 | */ 303 | virtual void logHbaseError( 304 | const char* operation, const string& remote_host, int remote_port, 305 | const string& row_key, int timeout_ms, const string& msg) { 306 | return; 307 | } 308 | 309 | /** 310 | * Log an RPC. Intended to be overridden by subclasses and used for 311 | * custom logging purposes. 312 | */ 313 | virtual void logHbaseOperation( 314 | const char* operation, const string& remote_host, int remote_port, 315 | const string& row_key, int timeout_ms) { 316 | return; 317 | } 318 | 319 | /** 320 | * Retrieve the list of region servers. 321 | * 322 | * @param output the output vector of host:port 323 | */ 324 | bool listRegionServers(HostPortVector* output); 325 | 326 | /** 327 | * Connect to the specified zookeeper instance. 328 | * 329 | * @returns true if successful 330 | */ 331 | bool connect(); 332 | 333 | /** 334 | * Use a new counter object for stats tracking; only valid before 335 | * connect() is called and can only be called once. A shared_ptr is 336 | * used as the caller may or may not keep ownership (ie, you may or 337 | * may not want to share counters among multiple clients). 338 | * 339 | * This can be called at any time, but it WILL segfault if you call 340 | * it while another thread is using the client, so serialization is 341 | * up to the caller (typically this would be done before many 342 | * queries are run). 343 | * 344 | * @returns nothing 345 | */ 346 | void setCounter(shared_ptr counter); 347 | 348 | /** 349 | * Determine if this client is healthy 350 | * 351 | * A client is unhealthy in general if it has had protocol-level 352 | * errors when conneting to the root region server. 353 | * 354 | * @returns true if healthy 355 | */ 356 | bool isHealthy() { return healthy_; } 357 | 358 | /** 359 | * Set the timeout for RPC operations. 360 | * 361 | * @param ms timeout, in milliseconds 362 | */ 363 | void setTimeout(int ms) { 364 | timeout_ms_ = ms; 365 | } 366 | 367 | int timeout() const { 368 | return timeout_ms_; 369 | } 370 | 371 | const HbaseClientTransport transport() const { 372 | return transport_; 373 | } 374 | 375 | // Some basic metadata APIs largely identical to their native 376 | // Hbase.thrift counterparts. 377 | 378 | /** 379 | * List tables in this Hbase cluster. 380 | * 381 | * @param output where the results are stored. 382 | */ 383 | void listTables(vector *output); 384 | 385 | /** 386 | * Get the column descriptors for a table. 387 | * 388 | * @param output where the results are stored. 389 | */ 390 | void getColumnDescriptors(map *output, 391 | const string& table); 392 | 393 | /** 394 | * Create a table. 395 | * 396 | * @param table the name of the table to create 397 | * @param columns a vector of ColumnDescriptor describing the table 398 | */ 399 | void createTable(const string& table, 400 | const vector& columns); 401 | 402 | /** 403 | * Delete (drop) a table. 404 | * 405 | * @param table the name of the table to drop 406 | */ 407 | void deleteTable(const string& table); 408 | 409 | /** 410 | * Get table status (enabled or disabled) 411 | * 412 | * @param table the name of the table to drop 413 | */ 414 | bool isTableEnabled(const string& table); 415 | 416 | /** 417 | * Enable a table. 418 | * 419 | * @param table the name of the table to enable 420 | */ 421 | void enableTable(const string& table); 422 | 423 | /** 424 | * Disable a table. 425 | * 426 | * @param table the name of the table to disable 427 | */ 428 | void disableTable(const string& table); 429 | 430 | /** 431 | * List regions for a table. 432 | * 433 | * @param output where the results are stored. 434 | * @param table name of the table 435 | */ 436 | void listTableRegions(vector *output, const string& table); 437 | 438 | /** 439 | * Fetch a single row (point lookup) 440 | * 441 | * @param output where the results are stored. 442 | * @param table name of the table 443 | * @param key the rowkey to fetch 444 | */ 445 | void getRow(vector* output, 446 | const string& table, const string& key); 447 | 448 | /** 449 | * Fetch the specified columns of a single row (point lookup) 450 | * 451 | * @param output where the results are stored. 452 | * @param table name of the table 453 | * @param key the rowkey to fetch 454 | * @param columns vector of the columns to fetch 455 | */ 456 | void getRowWithColumns(vector* output, 457 | const string& table, const string& key, 458 | const vector& column_specifiers); 459 | 460 | /** 461 | * Fetch the specified columns of a single row (point lookup) with a 462 | * specified upper bound on the timestamp 463 | * 464 | * @param output where the results are stored. 465 | * @param table name of the table 466 | * @param key the rowkey to fetch 467 | * @param columns vector of the columns to fetch 468 | * @param timestamp upper bound for the timestamp to fetch 469 | */ 470 | void getRowWithColumnsTs(vector* output, 471 | const string& table, const string& key, 472 | const vector& column_specifiers, 473 | const int64_t timestamp); 474 | 475 | /** 476 | * Fetch multiple rows in a given table (point lookup). 477 | * 478 | * @param output where the results are stored. 479 | * @param output_by_key a map of pointers to TRowResults (see below) 480 | * @param table name of the table 481 | * @param keys the rowkeys to fetch 482 | * @returns true if all region servers could be queried successfully 483 | * 484 | * output_by_key is a peculiar parameter. Unlike output, it can be 485 | * null'd. If it is not null, then it will map queried keys to 486 | * individual entries in the output vector. If a key is present and 487 | * the value is NULL, then the row had no result (ie, wasn't present 488 | * in hbase). If a key queried wasn't present in the map at all, it 489 | * indicates there was an error and no information is known about 490 | * the row (ie, it may or may not exist in hbase). This case only 491 | * occurs if getRows returns false. 492 | */ 493 | bool getRows(vector* output, 494 | unordered_map* output_by_key, 495 | const string& table, 496 | const vector& keys); 497 | 498 | bool getRowsWithColumns(vector* output, 499 | unordered_map* output_by_key, 500 | const string& table, 501 | const vector& keys, 502 | const vector& columns); 503 | /** 504 | * Create a scanner to return multiple rows. 505 | * 506 | * Get a scanner for the specified columns of the specified table, 507 | * starting at start_row. column_specifiers is a vector of column 508 | * names (typically "family:column") or regular expressions used to 509 | * select columns. TODO(chip): once the Thrift API supports the 510 | * new, non-colon delimited method of specifying columns, use that 511 | * here. 512 | * 513 | * @param table name of the table 514 | * @param start_row where the scan starts 515 | * @param column_specifiers which columns to return 516 | * 517 | * @returns a pointer to the scanner; caller must free it. 518 | */ 519 | Scanner* getScanner(const string& table, 520 | const string& start_row, 521 | const vector& column_specifiers); 522 | 523 | /** 524 | * Change a specific row in a table. 525 | * 526 | * Given a table and row, change the cells as specified in 527 | * mutations. 528 | * 529 | * @param table the table 530 | * @param row the row 531 | * @param mutations vector of columns, values, and delete flags to change 532 | * @param transaction_annotation free-form text annotating the transaction 533 | * 534 | * @returns true on success, false on failure 535 | */ 536 | bool mutateRow(const string& table, const string& row, 537 | const vector& mutations); 538 | 539 | bool mutateRow(const string& table, const string& row, 540 | const vector& mutations, 541 | const string& transaction_annotation); 542 | 543 | /** 544 | * Change a specific row in a table and set the timestamp. 545 | * 546 | * Given a table and row, change the cells as specified in 547 | * mutations. 548 | * 549 | * @param table the table 550 | * @param row the row 551 | * @param mutations vector of columns, values, and delete flags to change 552 | * @param timestamp timestamp to set for all affected cells 553 | * @param transaction_annotation free-form text annotating the transaction 554 | * 555 | * @returns true on success, false on failure 556 | */ 557 | bool mutateRowTs(const string& table, const string& row, 558 | const vector& mutations, 559 | const int64_t timestamp); 560 | 561 | bool mutateRowTs(const string& table, const string& row, 562 | const vector& mutations, 563 | const int64_t timestamp, 564 | const string& transaction_annotation); 565 | 566 | /** 567 | * Change a set of rows in the specified table. 568 | * 569 | * Given a table and a set of rows (and, for each row, a set of 570 | * columns) to change, modify the rows as appropriate. This is 571 | * *NOT* transactional across rows. Updates are batched per region 572 | * server and should be modestly efficient. 573 | * 574 | * @param table the table 575 | * @param rowBatches vector of BatchMutation operations 576 | * @param transaction_annotation free-form text annotating the transaction 577 | * 578 | * @returns true on success, false on failure of any of the many updates 579 | */ 580 | bool mutateRows(const string& table, const vector& rowBatches); 581 | 582 | bool mutateRows(const string& table, const vector& rowBatches, 583 | const string& transaction_annotation); 584 | 585 | /** 586 | * Change a set of rows in the specified table, setting the timestamp. 587 | * 588 | * Same as mutateRows, but also sets the timestamp to the specified value. 589 | * 590 | * @param table the table 591 | * @param rowBatches vector of BatchMutation operations 592 | * @param transaction_annotation free-form text annotating the transaction 593 | * 594 | * @returns true on success, false on failure of any of the many updates 595 | */ 596 | bool mutateRowsTs(const string& table, 597 | const vector& rowBatches, 598 | const int64_t timestamp); 599 | 600 | bool mutateRowsTs(const string& table, 601 | const vector& rowBatches, 602 | const int64_t timestamp, 603 | const string& transaction_annotation); 604 | 605 | /** 606 | * Atomically increment a given value. 607 | * 608 | * @param table the table 609 | * @param row the row 610 | * @param column the column 611 | * @param delta the amount to change the column by 612 | * 613 | * @returns the new value of the cell 614 | */ 615 | int64_t atomicIncrement(const string& table, const string& row, 616 | const string& column, int64_t delta); 617 | 618 | /** 619 | * Delete a specific column of a given row in the specified table. 620 | * 621 | * @param table the table 622 | * @param row the row 623 | * @param column the column 624 | * 625 | * @returns true on success, false on failure 626 | */ 627 | bool deleteAll(const string& table, const string& row, const string& column); 628 | 629 | /** 630 | * Like deleteAll but only deletes cells with timestamps less than 631 | * the specified timestamp. 632 | * 633 | * @param table the table 634 | * @param row the row 635 | * @param column the column 636 | * @param timestamp delete all timestamps up to and including this 637 | * 638 | * @returns true on success, false on failure 639 | */ 640 | bool deleteAllTs(const string& table, const string& row, 641 | const string& column, const int64_t timestamp); 642 | 643 | /** 644 | * Delete a specific row in the specified table. 645 | * 646 | * @param table the table 647 | * @param row the row 648 | * @param transaction_annotation free-form text annotating the transaction 649 | * 650 | * @returns true on success, false on failure 651 | */ 652 | bool deleteAllRow(const string& table, const string& row); 653 | 654 | bool deleteAllRow(const string& table, const string& row, 655 | const string& transaction_annotation); 656 | 657 | /** 658 | * Like deleteAllRow, but only deletes cells with a timestamp less 659 | * than or equal to the specified timestamp. 660 | * 661 | * @param table the table 662 | * @param row the row 663 | * @param timestamp delete all timestamps up to and including this 664 | * 665 | * @returns true on success, false on failure 666 | */ 667 | bool deleteAllRowTs(const string& table, const string& row, 668 | const int64_t timestamp); 669 | 670 | // Indicate a scanner is finished using the connection. 671 | void releaseConnection(NativeHbaseConnection* connection) { 672 | connection_pool_.release(connection); 673 | } 674 | 675 | CounterBase* getCounters() const { 676 | return stats_counters_.get(); 677 | } 678 | 679 | const string hosts() const { 680 | return hosts_; 681 | } 682 | 683 | // The attribute interpreted by the HBase region server as the 684 | // transaction annotation (aka wormhole comment). 685 | static const char* kWormholeHlogAttribute; 686 | 687 | protected: 688 | // A helper function to expand specified_hosts into a 689 | // comma-separated list of host:port's corresponding to the 690 | // zookeeper instances we should contact. The host:port's are also 691 | // placed into host_ports as pair. 692 | // 693 | // The intention is that specified_hosts could be anything, either 694 | // directly the list of hosts, or some convenient string that 695 | // subclasses might look up (say, SMC inside of Facebook). 696 | virtual bool expandConnectHosts(const string& specified_hosts, 697 | string* host_port_string, 698 | HostPortVector* host_ports); 699 | 700 | private: 701 | // Query zookeeper for our root region server and initialize root_conn_. 702 | bool initRootConnection(); 703 | 704 | // Called by Scanner when scan() is invoked. 705 | RegionScanner beginScanInternal(const string& table, 706 | const vector& columns, 707 | const string& start_row, 708 | const string& end_row); 709 | 710 | // Fetch a chunk of rows into the Scanner's internal buffer. 711 | bool getNextRows(vector* output, 712 | RegionScanner region_scanner, 713 | int num_rows); 714 | 715 | // Given a table and row, find the RegionServerInfo containing the 716 | // region; this region server is contacted directly for fetching rows. 717 | shared_ptr findRegion( 718 | const string& table, const string& row, 719 | bool ignore_cache); 720 | 721 | // Given a region server, return (or create) a NativeHbaseConnection 722 | // that communicates with it. The returned connection must 723 | // eventually be sent to releaseConnection. 724 | NativeHbaseConnection* connectionForRegionServer( 725 | const RegionServerInfo* region_server_info); 726 | 727 | // Helper that combined the above two functions. The returned 728 | // connection must eventually be sent to releaseConnection. 729 | NativeHbaseConnection* findConnection(const string& table, 730 | const string& row, 731 | bool ignore_cache) { 732 | return connectionForRegionServer( 733 | findRegion(table, row, ignore_cache).get()); 734 | } 735 | 736 | // A callable intended to be invoked per region server with keys 737 | // relative to that region server that populates a result set. 738 | // Hides the difference between getRows and getRowsWithColumns, 739 | // basically. 740 | typedef std::function&, 742 | const string&, 743 | const vector& keys)> GetRowsHelperOp; 744 | 745 | // A helper function for getRows and getRowsWithColumns; it invokes 746 | // the given GetRowsHelperOp against each region server with a 747 | // subset of the keys that correspond to that region server. 748 | bool getRowsHelper(vector* output, 749 | unordered_map* output_by_key, 750 | const string& table, 751 | const vector& keys, 752 | const char* opname, 753 | GetRowsHelperOp op); 754 | 755 | 756 | 757 | /** 758 | * Connect to the specified zookeeper instance without acquiring the lock. 759 | * This method has to be called with root_conn_mutex_ locked. 760 | * 761 | * @returns true if successful 762 | */ 763 | bool connectUnproteced(); 764 | 765 | bool healthy_; 766 | HostPortVector host_ports_; 767 | const string hosts_; // the hosts of the governing zookeeper 768 | system_clock::time_point last_zookeeper_connect_time_; 769 | zhandle_t* zh_; // zookeeper handle 770 | HbaseClientTransport transport_; // transport for this client 771 | 772 | std::mutex root_conn_mutex_; // see top-level comment about mutex order 773 | // client for the initial regions server; this client does not get 774 | // copied into our connection pool so as to not congest metadata 775 | // operations with queries. 776 | unique_ptr root_conn_; 777 | 778 | std::atomic num_regions_; // internal counter of regions loaded 779 | std::mutex regions_mutex_; // see top comment about mutex order 780 | TableMap regions_; // map of table_name to TableRegions 781 | 782 | int timeout_ms_; // timeout for RPCs, in ms 783 | int thrift_port_; // Port used for all thrift communication 784 | std::mt19937 rng_; // RNG, use must be protected by 785 | // root_conn_mutex_ 786 | 787 | // Connection pool for our connections 788 | HbaseConnectionPool connection_pool_; 789 | 790 | mutable shared_ptr stats_counters_; 791 | 792 | friend class Scanner; 793 | friend class FacebookHbaseClient; 794 | template friend class ConnectionGroup; 795 | }; 796 | 797 | } } // namespaces 798 | 799 | #endif // HBASE_SRC_NATIVEHBASECLIENT_H_ 800 | -------------------------------------------------------------------------------- /hbase/configure.ac: -------------------------------------------------------------------------------- 1 | 2 | # -*- Autoconf -*- 3 | # Process this file with autoconf to produce a configure script. 4 | 5 | AC_PREREQ(2.59) 6 | AC_INIT(nhc, 0.1, folly@fb.com) 7 | AC_CONFIG_SRCDIR([NativeHbaseClient.h]) 8 | AC_CONFIG_AUX_DIR([build-aux]) 9 | AX_PREFIX_CONFIG_H([nhc-config.h], [nhc], [nhc-config.h]) 10 | AC_CONFIG_HEADERS([nhc-config.h]) 11 | AC_DEFINE([OPEN_SOURCE], [1], [Always defined for the open source release]) 12 | 13 | AM_INIT_AUTOMAKE([foreign dist-bzip2]) 14 | 15 | AC_CONFIG_MACRO_DIR([m4]) 16 | 17 | AC_PROG_INSTALL 18 | AM_PROG_LIBTOOL 19 | 20 | AC_LANG([C++]) 21 | 22 | # Checks for programs. 23 | AC_PROG_CXX 24 | AC_PROG_CC 25 | AC_CXX_COMPILE_STDCXX_0X 26 | 27 | # Checks for libraries. 28 | AC_CHECK_LIB([glog],[openlog],[],[AC_MSG_ERROR( 29 | [Please install google-glog library])]) 30 | AC_CHECK_LIB([folly],[openlog],[],[AC_MSG_ERROR( 31 | [Please install the Facebook folly library])]) 32 | AC_CHECK_LIB([gflags],[getenv],[],[AC_MSG_ERROR( 33 | [Please install google-gflags library])]) 34 | AC_CHECK_LIB([gtest],[getenv],[],[AC_MSG_ERROR( 35 | [Please install google-gtest library])]) 36 | AC_CHECK_LIB([zookeeper_mt],[getenv],[],[AC_MSG_ERROR( 37 | [Please install zookeeper library])]) 38 | 39 | # Checks for header files. 40 | AC_CHECK_HEADER(folly/Likely.h, [], [AC_MSG_ERROR( 41 | [Couldn't find folly, please download from https://github.com/facebook/folly] 42 | )], []) 43 | AC_CHECK_HEADER(zookeeper/zookeeper.h, [], [AC_MSG_ERROR( 44 | [Couldn't find zookeeper, please download the C client library 45 | from http://zookeeper.apache.org/releases.html])], []) 46 | 47 | PKG_CHECK_MODULES(THRIFT, "thrift", [], [AC_MSG_ERROR( 48 | [Couldn't find thrift, please download from \ 49 | http://thrift.apache.org/])], []) 50 | 51 | 52 | AC_PATH_PROG(THRIFT_EXEC_PATH, "thrift") 53 | 54 | CXX_FLAGS="-Wall -Wno-sign-compare -Wno-unused-variable -Winvalid-pch -Wno-deprecated -Woverloaded-virtual -Wno-return-type -Werror" 55 | if test "$ac_cv_cxx_compile_cxx0x_cxx" = yes; then 56 | CXX_FLAGS="-std=c++0x" 57 | fi 58 | if test "$ac_cv_cxx_compile_cxx0x_gxx" = yes; then 59 | CXX_FLAGS="-std=gnu++0x" 60 | fi 61 | 62 | AC_SUBST(AM_CXXFLAGS, '-I../$(top_srcdir)'" $CXX_FLAGS") 63 | 64 | AM_CONDITIONAL([HAVE_STD_THREAD], [test "$ac_cv_header_features" = "yes"]) 65 | AM_CONDITIONAL([HAVE_X86_64], [test "$build_cpu" = "x86_64"]) 66 | 67 | # Output 68 | AC_CONFIG_FILES([Makefile]) 69 | AC_OUTPUT 70 | -------------------------------------------------------------------------------- /hbase/hbc.cpp: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | #include "folly/String.h" 17 | 18 | #include 19 | #include 20 | #include 21 | #include 22 | 23 | #include 24 | #include 25 | 26 | #include 27 | 28 | #include "folly/Logging.h" 29 | #include "folly/Range.h" 30 | #include "folly/Conv.h" 31 | #include "folly/String.h" 32 | #include "hbase/HbcExtensions.h" 33 | #include "hbase/NativeHbaseClient.h" 34 | 35 | const char* usage = 36 | "Usage: hbc [params] command [command params]\n" 37 | "\n" 38 | "Note: all commands rely on either the --zookeeper option or the \n" 39 | "HBC_ZOOKEEPER environment variable, which contains the ZooKeeper \n" 40 | "instance where the HBase cluster stores its state.\n" 41 | "\n" 42 | "For more details, see:\n" 43 | " https://www.intern.facebook.com/intern/wiki/index.php/HbaseClientTool\n"; 44 | 45 | using namespace facebook; 46 | using namespace std; 47 | 48 | using folly::backslashify; 49 | using folly::humanify; 50 | using folly::StringPiece; 51 | using folly::to; 52 | 53 | DEFINE_string(zookeeper, getenv("HBC_ZOOKEEPER") ? : "", 54 | "SMC tier containing the zookeeper hosts, or directly a comma " 55 | "separated list of host:ports for our zookeeper instance"); 56 | DEFINE_string(start_row, "", "Row to begin with in 'scan' mode"); 57 | DEFINE_string(end_row, "", "Row to end with in 'scan' mode"); 58 | DEFINE_string(column, "", "Column to print (regex)"); 59 | DEFINE_int32(num_scan_rows, -1, "Max number of rows to return when scanning; " 60 | "-1 for all rows"); 61 | DEFINE_string(timestamp, "", "Timestamp to restrict returned rows to (uint64)"); 62 | 63 | DEFINE_bool(print_timestamp, false, "Include timestamps in output"); 64 | DEFINE_bool(bypass_special_row_display, false, "always display row results as " 65 | "key/value rather than using extended output processing"); 66 | DEFINE_bool(smart_row_interpretation, true, "if set, hbc will attempt to " 67 | "guess row key meaning based on prefixes rather than literally; " 68 | "for instance, keys starting with 0x will be interpreted as hex"); 69 | DECLARE_int32(hbase_port); 70 | DEFINE_bool(disable_wal, false, 71 | "whether Mutation write-ahead-log will be disabled on the server"); 72 | 73 | namespace facebook { namespace hbase { 74 | const string smartKeyGuess(const string& input) { 75 | if (!FLAGS_smart_row_interpretation) return input; 76 | 77 | string ret(input); 78 | if (boost::starts_with(input, "0x")) { 79 | if (!folly::unhexlify(string(input, 2), ret)) { 80 | LOG(FATAL) << "Invalid hex string: " << input; 81 | } 82 | return ret; 83 | } 84 | // TODO(chip): also support backslash encoding, perhaps binary 85 | // encoding as well? 86 | 87 | return ret; 88 | } 89 | 90 | static vector row_printers; 91 | bool invokeRowPrinter(const TRowResult* row) { 92 | if (FLAGS_bypass_special_row_display) return false; 93 | 94 | for (auto& rp : row_printers) { 95 | if (rp(row)) return true; 96 | } 97 | return false; 98 | } 99 | 100 | class RegionsCommand : public Subcommand { 101 | public: 102 | RegionsCommand() : Subcommand("regions") { } 103 | 104 | void Usage(const string& prefix) { 105 | cout << prefix << "regions [
... ]" << endl; 106 | } 107 | 108 | int Execute(NativeHbaseClient* client, const vector& args) { 109 | vector tables; 110 | 111 | client->listTables(&tables); 112 | for (int i = 0; i < tables.size(); ++i) { 113 | if (args.size() > 0 && 114 | find(args.begin(), args.end(), tables[i]) == args.end()) { 115 | continue; 116 | } 117 | cout << "Table: " << tables[i] << endl; 118 | 119 | vector regions; 120 | client->listTableRegions(®ions, tables[i]); 121 | for (int j = 0; j < regions.size(); ++j) { 122 | cout << " Region " << backslashify(regions[j].name) << endl 123 | << " Host : " << regions[j].serverName << ":" << regions[j].port 124 | << endl 125 | << " Start: " << humanify(regions[j].startKey) << endl 126 | << " End : " << humanify(regions[j].endKey) << endl; 127 | } 128 | cout << endl; 129 | } 130 | 131 | return 0; 132 | } 133 | }; 134 | 135 | class TablesCommand : public Subcommand { 136 | public: 137 | TablesCommand() : Subcommand("tables") { } 138 | void Usage(const string& prefix) { 139 | cout << prefix << "tables" << endl; 140 | } 141 | int Execute(NativeHbaseClient* client, const vector& args) { 142 | vector tables; 143 | 144 | client->listTables(&tables); 145 | for (int i = 0; i < tables.size(); ++i) { 146 | cout << "Table: " << tables[i] << endl; 147 | } 148 | 149 | return 0; 150 | } 151 | }; 152 | 153 | class RegionServersCommand : public Subcommand { 154 | public: 155 | RegionServersCommand() : Subcommand("region_servers") { } 156 | void Usage(const string& prefix) { 157 | cout << prefix << "region_servers" << endl; 158 | } 159 | int Execute(NativeHbaseClient* client, const vector& args) { 160 | vector> region_servers; 161 | 162 | client->listRegionServers(®ion_servers); 163 | cout << "Region servers:" << endl; 164 | for (int i = 0; i < region_servers.size(); ++i) { 165 | cout << " " << region_servers[i].first << ":" 166 | << region_servers[i].second << endl; 167 | } 168 | 169 | return 0; 170 | } 171 | }; 172 | 173 | // A relatively simple routine that attempts to format a cell for 174 | // display. For printable strings, it just returns the string in 175 | // quotes; for 1, 2, 4, or 8 byte unprintable strings, it turns them 176 | // into the appropriate type of integer, performs an endian swap, and 177 | // returns that. Otherwise, unprintable strings are hexdumped. 178 | static string FormatCellValue(const TCell& cell) { 179 | const string& input = cell.value; 180 | bool has_binary = false; 181 | for (unsigned char c : input) { 182 | if (!isprint(c)) { 183 | has_binary = true; 184 | break; 185 | } 186 | } 187 | if (!has_binary) { 188 | return '"' + input + '"'; 189 | } 190 | 191 | string output; 192 | if (input.size() == 1) { 193 | int8_t value; 194 | memcpy(&value, &input[0], 1); 195 | output = folly::stringPrintf("int8:%d", value); 196 | } else if (input.size() == 2) { 197 | int16_t value; 198 | memcpy(&value, &input[0], 2); 199 | output = folly::stringPrintf("int16:%d", htons(value)); 200 | } else if (input.size() == 4) { 201 | int32_t value; 202 | memcpy(&value, &input[0], 4); 203 | output = folly::stringPrintf("int32_t:%d", htonl(value)); 204 | } else if (input.size() == 8) { 205 | int64_t value; 206 | memcpy(&value, &input[0], 8); 207 | output = folly::stringPrintf("int64_t:%lu", htonll(value)); 208 | } else { 209 | CHECK(folly::hexlify(cell.value, output)); 210 | output = "0x" + output; 211 | } 212 | 213 | if (FLAGS_print_timestamp) { 214 | output += folly::stringPrintf(" (%lu)", cell.timestamp); 215 | } 216 | 217 | return output; 218 | } 219 | 220 | class GetCommand : public Subcommand { 221 | public: 222 | GetCommand() : Subcommand("get") { } 223 | void Usage(const string& prefix) { 224 | cout << prefix << "get
" << endl; 225 | } 226 | int Execute(NativeHbaseClient* client, const vector& args) { 227 | if (args.size() < 2) { 228 | throw UsageException(); 229 | } 230 | 231 | if (!FLAGS_timestamp.empty() && FLAGS_column.empty()) { 232 | LOG(ERROR) << "Must include --column when using --timestamp"; 233 | throw UsageException(); 234 | } 235 | 236 | const string table(extractTable(client, args, 0)); 237 | const string row_key(smartKeyGuess(args[1])); 238 | 239 | vector rows; 240 | if (FLAGS_column.empty()) { 241 | client->getRow(&rows, table, row_key); 242 | } else { 243 | vector columns; 244 | columns.push_back(FLAGS_column); 245 | if (!FLAGS_timestamp.empty()) { 246 | uint64_t timestamp; 247 | timestamp = to(FLAGS_timestamp); 248 | client->getRowWithColumnsTs(&rows, table, row_key, columns, timestamp); 249 | } else { 250 | client->getRowWithColumns(&rows, table, row_key, columns); 251 | } 252 | } 253 | for (TRowResult &row : rows) { 254 | if (!invokeRowPrinter(&row)) { 255 | for (auto& kv : row.columns) { 256 | cout << humanify(kv.first) << ": " << humanify(kv.second.value) 257 | << endl; 258 | } 259 | } 260 | } 261 | 262 | return 0; 263 | } 264 | }; 265 | 266 | class SetCommand : public Subcommand { 267 | public: 268 | SetCommand() : Subcommand("set") { } 269 | void Usage(const string& prefix) { 270 | cout << prefix 271 | << "set [--disable_wal]
column=valuue column=value ..." 272 | << endl; 273 | } 274 | int Execute(NativeHbaseClient* client, const vector& args) { 275 | if (args.size() < 3) { 276 | throw UsageException(); 277 | } 278 | 279 | const string table(extractTable(client, args, 0)); 280 | const string row_key(smartKeyGuess(args[1])); 281 | vector mutations; 282 | 283 | bool write_to_wal = !FLAGS_disable_wal; 284 | for (int i = 2; i < args.size(); ++i) { 285 | vector assignment; 286 | folly::split("=", args[i], assignment); 287 | if (assignment.size() != 2) { 288 | LOG(ERROR) << "Invalid assignment: " << args[i]; 289 | throw UsageException(); 290 | } 291 | 292 | mutations.resize(mutations.size() + 1); 293 | BatchMutation &batch = mutations.back(); 294 | batch.row = row_key; 295 | 296 | Mutation mutation; 297 | mutation.column = assignment[0]; 298 | mutation.value = assignment[1]; 299 | mutation.isDelete = false; 300 | mutation.writeToWAL = write_to_wal; 301 | batch.mutations.push_back(mutation); 302 | } 303 | if (!client->mutateRows(table, mutations)) { 304 | LOG(ERROR) << "One or more mutation failed"; 305 | } 306 | 307 | return 0; 308 | } 309 | }; 310 | 311 | class DeleteCommand : public Subcommand { 312 | public: 313 | DeleteCommand() : Subcommand("delete") { } 314 | void Usage(const string& prefix) { 315 | cout << prefix << "delete
[column column ...]" 316 | << endl; 317 | } 318 | int Execute(NativeHbaseClient* client, const vector& args) { 319 | if (args.size() < 2) { 320 | throw UsageException(); 321 | } 322 | 323 | const string table(extractTable(client, args, 0)); 324 | const string row_key(smartKeyGuess(args[1])); 325 | 326 | // Use batch mutation just to exercise the more complex codepath. 327 | vector mutations; 328 | 329 | if (args.size() == 2) { 330 | if (!client->deleteAllRow(table, row_key)) { 331 | LOG(ERROR) << "Unable to delete entire row: " << row_key; 332 | } 333 | } else { 334 | for (int i = 2; i < args.size(); ++i) { 335 | if (!client->deleteAll(table, row_key, args[i])) { 336 | LOG(ERROR) << "Delete failed, row: " << row_key << " cell: " 337 | << args[i]; 338 | } 339 | } 340 | } 341 | 342 | return 0; 343 | } 344 | }; 345 | 346 | class ScanCommand : public Subcommand { 347 | public: 348 | ScanCommand() : Subcommand("scan") { } 349 | void Usage(const string& prefix) { 350 | cout << prefix << "scan [ --start_row R ] [ --end_row R ] " 351 | << "[ --column C ] [--scan_limit N]
" << endl; 352 | } 353 | int Execute(NativeHbaseClient* client, const vector& args) { 354 | if (args.size() < 1) { 355 | throw UsageException(); 356 | } 357 | 358 | const string table(extractTable(client, args, 0)); 359 | vector columns; 360 | if (FLAGS_column.size()) { 361 | columns.push_back(FLAGS_column); 362 | } 363 | std::unique_ptr scanner( 364 | client->getScanner(table, smartKeyGuess(FLAGS_start_row), columns)); 365 | 366 | if (FLAGS_end_row.size()) { 367 | scanner->setEndRow(smartKeyGuess(FLAGS_end_row)); 368 | } 369 | scanner->scan(); 370 | TRowResult *result; 371 | int rows_remaining = FLAGS_num_scan_rows; 372 | while (scanner->next(&result) && rows_remaining != 0) { 373 | if (!invokeRowPrinter(result)) { 374 | cout << "Row " << humanify(result->row) << endl; 375 | Text column; 376 | TCell cell; 377 | for(auto& kv : result->columns) { 378 | cout << " " << humanify(kv.first) << " - " 379 | << FormatCellValue(kv.second) << endl; 380 | } 381 | } 382 | --rows_remaining; 383 | } 384 | 385 | return 0; 386 | } 387 | }; 388 | 389 | class SchemaCommand : public Subcommand { 390 | public: 391 | SchemaCommand() : Subcommand("schema") { } 392 | void Usage(const string& prefix) { 393 | cout << prefix << "schema
" << endl; 394 | } 395 | int Execute(NativeHbaseClient* client, const vector& args) { 396 | if (args.size() != 1) { 397 | throw UsageException(); 398 | } 399 | 400 | const string table(extractTable(client, args, 0)); 401 | map columns; 402 | client->getColumnDescriptors(&columns, table); 403 | 404 | cout << "# Table schema for " << table << endl; 405 | time_t now = time(NULL); 406 | char buf[26]; 407 | CHECK(ctime_r(&now, buf)); 408 | cout << "# Dumped on " << buf; 409 | cout << endl; 410 | 411 | #define FIELD_PRINT(f) cout << " " #f ": " << kv.second.f << endl 412 | for (auto& kv : columns) { 413 | cout << "<" << endl; 414 | FIELD_PRINT(name); 415 | FIELD_PRINT(maxVersions); 416 | FIELD_PRINT(compression); 417 | FIELD_PRINT(inMemory); 418 | FIELD_PRINT(bloomFilterType); 419 | FIELD_PRINT(bloomFilterVectorSize); 420 | FIELD_PRINT(bloomFilterNbHashes); 421 | FIELD_PRINT(blockCacheEnabled); 422 | FIELD_PRINT(timeToLive); 423 | cout << ">" << endl; 424 | } 425 | #undef FIELD_PRINT 426 | 427 | return 0; 428 | } 429 | }; 430 | 431 | class CreateCommand : public Subcommand { 432 | public: 433 | CreateCommand() : Subcommand("create") { } 434 | void Usage(const string& prefix) { 435 | cout << prefix << "create
/path/to/schema" << endl; 436 | cout << prefix << "create
(schema read from stdin)" << endl; 437 | } 438 | bool parseLine(ColumnDescriptor* column, const string& line) { 439 | size_t pos = line.find(":"); 440 | if (pos == 0 || pos == string::npos) { 441 | return false; 442 | } 443 | string field(line, 0, pos); 444 | ++pos; 445 | while (pos < line.length() && isspace(line[pos])) { 446 | ++pos; 447 | } 448 | string value(line, pos); 449 | 450 | if (field == "name") { 451 | column->name = value; 452 | } else if (field == "maxVersions") { 453 | column->maxVersions = to(value); 454 | } else if (field == "compression") { 455 | column->compression = value; 456 | } else if (field == "inMemory") { 457 | column->inMemory = to(value); 458 | } else if (field == "bloomFilterType") { 459 | column->bloomFilterType = value; 460 | } else if (field == "bloomFilterVectorSize") { 461 | column->bloomFilterVectorSize = to(value); 462 | } else if (field == "bloomFilterNbHashes") { 463 | column->bloomFilterNbHashes = to(value); 464 | } else if (field == "blockCacheEnabled") { 465 | column->blockCacheEnabled = to(value); 466 | } else if (field == "timeToLive") { 467 | column->timeToLive = to(value); 468 | } else { 469 | return false; 470 | } 471 | 472 | return true; 473 | } 474 | int Execute(NativeHbaseClient* client, const vector& args) { 475 | if (args.size() != 1 && args.size() != 2) { 476 | throw UsageException(); 477 | } 478 | 479 | const string table(args[0]); 480 | 481 | int fd; 482 | if (args.size() == 1) { 483 | fd = STDIN_FILENO; 484 | } 485 | else { 486 | fd = open(args.at(1).c_str(), O_RDONLY); 487 | if (fd < 0) { 488 | PLOG(FATAL) << "Unable to open file: " << args[1]; 489 | } 490 | } 491 | 492 | string contents(65536, '\0'); 493 | size_t size = read(fd, &contents[0], contents.size()); 494 | if (size <= 0 || size == contents.size()) { 495 | throw FatalException("Unable to read schema definition"); 496 | } 497 | contents.resize(size); 498 | 499 | vector lines; 500 | folly::split("\n", contents, lines); 501 | 502 | vector columns; 503 | bool in_column_definition = false; 504 | ColumnDescriptor current_column; 505 | for (auto raw_line : lines) { 506 | string line = raw_line.toString(); 507 | boost::trim(line); 508 | if (line.size() == 0) continue; 509 | if (line[0] == '#') continue; 510 | if (line == "<") { 511 | CHECK(!in_column_definition); 512 | in_column_definition = true; 513 | } else if (line == ">") { 514 | CHECK(in_column_definition); 515 | in_column_definition = false; 516 | CHECK(!current_column.name.empty()); 517 | columns.push_back(current_column); 518 | } else { 519 | CHECK(in_column_definition); 520 | if (!parseLine(¤t_column, line)) { 521 | return 1; 522 | } 523 | } 524 | } 525 | CHECK(!in_column_definition); 526 | 527 | client->createTable(table, columns); 528 | cout << "Create successful!" << endl; 529 | 530 | return 0; 531 | } 532 | }; 533 | 534 | class DeleteTableCommand : public Subcommand { 535 | public: 536 | DeleteTableCommand() : Subcommand("drop") { } 537 | void Usage(const string& prefix) { 538 | cout << prefix << "drop
" << endl; 539 | } 540 | int Execute(NativeHbaseClient* client, const vector& args) { 541 | if (args.size() != 1) { 542 | throw UsageException(); 543 | } 544 | 545 | const string table(extractTable(client, args, 0)); 546 | 547 | try { 548 | client->deleteTable(table); 549 | cout << "Table deleted." << endl; 550 | return 0; 551 | } 552 | catch (const IOError& e) { 553 | cout << "Unable to drop table: " << e.message << endl; 554 | } 555 | return 1; 556 | } 557 | }; 558 | 559 | class TableStatusCommand : public Subcommand { 560 | public: 561 | TableStatusCommand() : Subcommand("status") { } 562 | void Usage(const string& prefix) { 563 | cout << prefix << "status
" << endl; 564 | } 565 | int Execute(NativeHbaseClient* client, const vector& args) { 566 | if (args.size() != 1) { 567 | throw UsageException(); 568 | } 569 | 570 | const string table(extractTable(client, args, 0)); 571 | 572 | bool result = client->isTableEnabled(table); 573 | if (result) { 574 | cout << "Table is enabled." << endl; 575 | } else { 576 | cout << "Table is disabled." << endl; 577 | } 578 | 579 | return !result; 580 | } 581 | }; 582 | 583 | class EnableDisableTableCommand : public Subcommand { 584 | public: 585 | explicit EnableDisableTableCommand(bool enable) : 586 | Subcommand(enable ? "enable" : "disable"), 587 | enable_(enable) { } 588 | void Usage(const string& prefix) { 589 | if (enable_) { 590 | cout << prefix << "enable
" << endl; 591 | } else { 592 | cout << prefix << "disable
" << endl; 593 | } 594 | } 595 | int Execute(NativeHbaseClient* client, const vector& args) { 596 | if (args.size() != 1) { 597 | throw UsageException(); 598 | } 599 | 600 | const string table(extractTable(client, args, 0)); 601 | 602 | if (enable_) { 603 | client->enableTable(table); 604 | cout << "Table enabled." << endl; 605 | } else { 606 | client->disableTable(table); 607 | cout << "Table disabled." << endl; 608 | } 609 | 610 | return 0; 611 | } 612 | private: 613 | bool enable_; 614 | }; 615 | 616 | static vector extension_command_groups; 617 | void RegisterExtensionCommands(CommandGroup* group) { 618 | extension_command_groups.push_back(group); 619 | } 620 | 621 | static ConnectionMaker default_connection_maker = 622 | [](const string& zk) { return new NativeHbaseClient(zk); }; 623 | 624 | void RegisterConnectionMaker(ConnectionMaker maker) { 625 | static bool first_invocation = true; 626 | CHECK(first_invocation) 627 | << "Only one call to RegisterConnectionMaker is allowed; are you linking " 628 | << "incompatible hbc extensions?"; 629 | first_invocation = false; 630 | default_connection_maker = maker; 631 | } 632 | 633 | void RegisterSpecialRowPrinter(RowPrinter printer) { 634 | row_printers.push_back(printer); 635 | } 636 | 637 | } } // namespace facebook::hbase 638 | 639 | namespace h = facebook::hbase; 640 | 641 | void Usage(const vector& command_groups) { 642 | cout << usage << endl; 643 | if (!FLAGS_zookeeper.empty()) { 644 | cout << "Current ZooKeeper instance: " << FLAGS_zookeeper << endl; 645 | } 646 | else { 647 | cout << "No ZooKeeper instance set via command line or environment." 648 | << endl; 649 | } 650 | cout << endl; 651 | 652 | cout << "Subcommands:" << endl; 653 | for (h::CommandGroup* group : command_groups) { 654 | cout << " " << group->name << ": " << endl; 655 | for (h::Subcommand* command : group->commands) { 656 | command->Usage(" hbc "); 657 | } 658 | cout << endl; 659 | } 660 | } 661 | 662 | // This function is implemented externally; it is different for 663 | // Facebook to tie in our hooks but generally will register new 664 | // subfunctions and connection creation objects. 665 | namespace facebook { namespace hbase { 666 | void initHbcExtensions(); 667 | #if NHC_OPEN_SOURCE 668 | void initHbcExtensions() { } 669 | #endif 670 | } } 671 | 672 | int main(int argc, char **argv) { 673 | google::InstallFailureSignalHandler(); 674 | google::ParseCommandLineFlags(&argc, &argv, true); 675 | google::InitGoogleLogging(argv[0]); 676 | facebook::hbase::initHbcExtensions(); 677 | 678 | vector command_groups; 679 | 680 | h::CommandGroup* inspection = new h::CommandGroup("Table/Cell inspection"); 681 | inspection->addSubcommand(new h::TablesCommand()); 682 | inspection->addSubcommand(new h::SchemaCommand()); 683 | inspection->addSubcommand(new h::RegionsCommand()); 684 | inspection->addSubcommand(new h::RegionServersCommand()); 685 | command_groups.push_back(inspection); 686 | 687 | h::CommandGroup* data = new h::CommandGroup("Data manipulation"); 688 | data->addSubcommand(new h::ScanCommand()); 689 | data->addSubcommand(new h::GetCommand()); 690 | data->addSubcommand(new h::SetCommand()); 691 | data->addSubcommand(new h::DeleteCommand()); 692 | command_groups.push_back(data); 693 | 694 | h::CommandGroup* schema = new h::CommandGroup("Schema manipulation"); 695 | schema->addSubcommand(new h::CreateCommand()); 696 | schema->addSubcommand(new h::DeleteTableCommand()); 697 | schema->addSubcommand(new h::EnableDisableTableCommand(true)); 698 | schema->addSubcommand(new h::EnableDisableTableCommand(false)); 699 | schema->addSubcommand(new h::TableStatusCommand()); 700 | command_groups.push_back(schema); 701 | 702 | // Extension groups always come after core groups. 703 | std::copy(h::extension_command_groups.begin(), 704 | h::extension_command_groups.end(), 705 | std::back_inserter(command_groups)); 706 | 707 | if (argc < 2) { 708 | Usage(command_groups); 709 | return 1; 710 | } 711 | 712 | vector args(argc - 2); 713 | copy(&(argv[2]), &(argv[argc]), args.begin()); 714 | 715 | for (h::CommandGroup* group : command_groups) { 716 | for (h::Subcommand* command : group->commands) { 717 | if (command->name == argv[1]) { 718 | try { 719 | unique_ptr client( 720 | h::default_connection_maker(FLAGS_zookeeper)); 721 | if (!client->connect()) { 722 | cerr << "Unable to connect to zookeeper: " << FLAGS_zookeeper 723 | << endl; 724 | return 2; 725 | } 726 | return command->Execute(client.get(), args); 727 | } catch (const h::UsageException& e) { 728 | command->Usage("Error: Usage: hbc "); 729 | return 1; 730 | } catch (const h::FatalException& e) { 731 | cerr << e.what() << endl; 732 | return 2; 733 | } 734 | } 735 | } 736 | } 737 | 738 | Usage(command_groups); 739 | 740 | return 1; 741 | } 742 | 743 | -------------------------------------------------------------------------------- /hbase/m4/ac_cxx_compile_stdcxx_0x.m4: -------------------------------------------------------------------------------- 1 | # =========================================================================== 2 | # http://autoconf-archive.cryp.to/ac_cxx_compile_stdcxx_0x.html 3 | # =========================================================================== 4 | # 5 | # SYNOPSIS 6 | # 7 | # AC_CXX_COMPILE_STDCXX_0X 8 | # 9 | # DESCRIPTION 10 | # 11 | # Check for baseline language coverage in the compiler for the C++0x 12 | # standard. 13 | # 14 | # LAST MODIFICATION 15 | # 16 | # 2008-04-17 17 | # 18 | # COPYLEFT 19 | # 20 | # Copyright (c) 2008 Benjamin Kosnik 21 | # 22 | # Copying and distribution of this file, with or without modification, are 23 | # permitted in any medium without royalty provided the copyright notice 24 | # and this notice are preserved. 25 | 26 | AC_DEFUN([AC_CXX_COMPILE_STDCXX_0X], [ 27 | AC_CACHE_CHECK(if g++ supports C++0x features without additional flags, 28 | ac_cv_cxx_compile_cxx0x_native, 29 | [AC_LANG_SAVE 30 | AC_LANG_CPLUSPLUS 31 | AC_TRY_COMPILE([ 32 | template 33 | struct check 34 | { 35 | static_assert(sizeof(int) <= sizeof(T), "not big enough"); 36 | }; 37 | 38 | typedef check> right_angle_brackets; 39 | 40 | int a; 41 | decltype(a) b; 42 | 43 | typedef check check_type; 44 | check_type c; 45 | check_type&& cr = static_cast(c);],, 46 | ac_cv_cxx_compile_cxx0x_native=yes, ac_cv_cxx_compile_cxx0x_native=no) 47 | AC_LANG_RESTORE 48 | ]) 49 | 50 | AC_CACHE_CHECK(if g++ supports C++0x features with -std=c++0x, 51 | ac_cv_cxx_compile_cxx0x_cxx, 52 | [AC_LANG_SAVE 53 | AC_LANG_CPLUSPLUS 54 | ac_save_CXXFLAGS="$CXXFLAGS" 55 | CXXFLAGS="$CXXFLAGS -std=c++0x" 56 | AC_TRY_COMPILE([ 57 | template 58 | struct check 59 | { 60 | static_assert(sizeof(int) <= sizeof(T), "not big enough"); 61 | }; 62 | 63 | typedef check> right_angle_brackets; 64 | 65 | int a; 66 | decltype(a) b; 67 | 68 | typedef check check_type; 69 | check_type c; 70 | check_type&& cr = static_cast(c);],, 71 | ac_cv_cxx_compile_cxx0x_cxx=yes, ac_cv_cxx_compile_cxx0x_cxx=no) 72 | CXXFLAGS="$ac_save_CXXFLAGS" 73 | AC_LANG_RESTORE 74 | ]) 75 | 76 | AC_CACHE_CHECK(if g++ supports C++0x features with -std=gnu++0x, 77 | ac_cv_cxx_compile_cxx0x_gxx, 78 | [AC_LANG_SAVE 79 | AC_LANG_CPLUSPLUS 80 | ac_save_CXXFLAGS="$CXXFLAGS" 81 | CXXFLAGS="$CXXFLAGS -std=gnu++0x" 82 | AC_TRY_COMPILE([ 83 | template 84 | struct check 85 | { 86 | static_assert(sizeof(int) <= sizeof(T), "not big enough"); 87 | }; 88 | 89 | typedef check> right_angle_brackets; 90 | 91 | int a; 92 | decltype(a) b; 93 | 94 | typedef check check_type; 95 | check_type c; 96 | check_type&& cr = static_cast(c);],, 97 | ac_cv_cxx_compile_cxx0x_gxx=yes, ac_cv_cxx_compile_cxx0x_gxx=no) 98 | CXXFLAGS="$ac_save_CXXFLAGS" 99 | AC_LANG_RESTORE 100 | ]) 101 | 102 | if test "$ac_cv_cxx_compile_cxx0x_native" = yes || 103 | test "$ac_cv_cxx_compile_cxx0x_cxx" = yes || 104 | test "$ac_cv_cxx_compile_cxx0x_gxx" = yes; then 105 | AC_DEFINE(HAVE_STDCXX_0X,,[Define if g++ supports C++0x features. ]) 106 | else 107 | AC_MSG_ERROR([Could not find cxx0x support in g++]) 108 | fi 109 | ]) 110 | -------------------------------------------------------------------------------- /hbase/m4/ax_prefix_config.m4: -------------------------------------------------------------------------------- 1 | # =========================================================================== 2 | # http://www.gnu.org/software/autoconf-archive/ax_prefix_config_h.html 3 | # =========================================================================== 4 | # 5 | # SYNOPSIS 6 | # 7 | # AX_PREFIX_CONFIG_H [(OUTPUT-HEADER [,PREFIX [,ORIG-HEADER]])] 8 | # 9 | # DESCRIPTION 10 | # 11 | # This is a new variant from ac_prefix_config_ this one will use a 12 | # lowercase-prefix if the config-define was starting with a 13 | # lowercase-char, e.g. "#define const", "#define restrict", or "#define 14 | # off_t", (and this one can live in another directory, e.g. 15 | # testpkg/config.h therefore I decided to move the output-header to be the 16 | # first arg) 17 | # 18 | # takes the usual config.h generated header file; looks for each of the 19 | # generated "#define SOMEDEF" lines, and prefixes the defined name (ie. 20 | # makes it "#define PREFIX_SOMEDEF". The result is written to the output 21 | # config.header file. The PREFIX is converted to uppercase for the 22 | # conversions. 23 | # 24 | # Defaults: 25 | # 26 | # OUTPUT-HEADER = $PACKAGE-config.h 27 | # PREFIX = $PACKAGE 28 | # ORIG-HEADER, from AM_CONFIG_HEADER(config.h) 29 | # 30 | # Your configure.ac script should contain both macros in this order, and 31 | # unlike the earlier variations of this prefix-macro it is okay to place 32 | # the AX_PREFIX_CONFIG_H call before the AC_OUTPUT invokation. 33 | # 34 | # Example: 35 | # 36 | # AC_INIT(config.h.in) # config.h.in as created by "autoheader" 37 | # AM_INIT_AUTOMAKE(testpkg, 0.1.1) # makes #undef VERSION and PACKAGE 38 | # AM_CONFIG_HEADER(config.h) # prep config.h from config.h.in 39 | # AX_PREFIX_CONFIG_H(mylib/_config.h) # prep mylib/_config.h from it.. 40 | # AC_MEMORY_H # makes "#undef NEED_MEMORY_H" 41 | # AC_C_CONST_H # makes "#undef const" 42 | # AC_OUTPUT(Makefile) # creates the "config.h" now 43 | # # and also mylib/_config.h 44 | # 45 | # if the argument to AX_PREFIX_CONFIG_H would have been omitted then the 46 | # default outputfile would have been called simply "testpkg-config.h", but 47 | # even under the name "mylib/_config.h" it contains prefix-defines like 48 | # 49 | # #ifndef TESTPKG_VERSION 50 | # #define TESTPKG_VERSION "0.1.1" 51 | # #endif 52 | # #ifndef TESTPKG_NEED_MEMORY_H 53 | # #define TESTPKG_NEED_MEMORY_H 1 54 | # #endif 55 | # #ifndef _testpkg_const 56 | # #define _testpkg_const _const 57 | # #endif 58 | # 59 | # and this "mylib/_config.h" can be installed along with other 60 | # header-files, which is most convenient when creating a shared library 61 | # (that has some headers) where some functionality is dependent on the 62 | # OS-features detected at compile-time. No need to invent some 63 | # "mylib-confdefs.h.in" manually. :-) 64 | # 65 | # Note that some AC_DEFINEs that end up in the config.h file are actually 66 | # self-referential - e.g. AC_C_INLINE, AC_C_CONST, and the AC_TYPE_OFF_T 67 | # say that they "will define inline|const|off_t if the system does not do 68 | # it by itself". You might want to clean up about these - consider an 69 | # extra mylib/conf.h that reads something like: 70 | # 71 | # #include 72 | # #ifndef _testpkg_const 73 | # #define _testpkg_const const 74 | # #endif 75 | # 76 | # and then start using _testpkg_const in the header files. That is also a 77 | # good thing to differentiate whether some library-user has starting to 78 | # take up with a different compiler, so perhaps it could read something 79 | # like this: 80 | # 81 | # #ifdef _MSC_VER 82 | # #include 83 | # #else 84 | # #include 85 | # #endif 86 | # #ifndef _testpkg_const 87 | # #define _testpkg_const const 88 | # #endif 89 | # 90 | # LICENSE 91 | # 92 | # Copyright (c) 2008 Guido U. Draheim 93 | # Copyright (c) 2008 Marten Svantesson 94 | # Copyright (c) 2008 Gerald Point 95 | # 96 | # This program is free software; you can redistribute it and/or modify it 97 | # under the terms of the GNU General Public License as published by the 98 | # Free Software Foundation; either version 3 of the License, or (at your 99 | # option) any later version. 100 | # 101 | # This program is distributed in the hope that it will be useful, but 102 | # WITHOUT ANY WARRANTY; without even the implied warranty of 103 | # MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General 104 | # Public License for more details. 105 | # 106 | # You should have received a copy of the GNU General Public License along 107 | # with this program. If not, see . 108 | # 109 | # As a special exception, the respective Autoconf Macro's copyright owner 110 | # gives unlimited permission to copy, distribute and modify the configure 111 | # scripts that are the output of Autoconf when processing the Macro. You 112 | # need not follow the terms of the GNU General Public License when using 113 | # or distributing such scripts, even though portions of the text of the 114 | # Macro appear in them. The GNU General Public License (GPL) does govern 115 | # all other use of the material that constitutes the Autoconf Macro. 116 | # 117 | # This special exception to the GPL applies to versions of the Autoconf 118 | # Macro released by the Autoconf Archive. When you make and distribute a 119 | # modified version of the Autoconf Macro, you may extend this special 120 | # exception to the GPL to apply to your modified version as well. 121 | 122 | #serial 11 123 | 124 | AC_DEFUN([AX_PREFIX_CONFIG_H],[dnl 125 | AC_PREREQ([2.62]) 126 | AC_BEFORE([AC_CONFIG_HEADERS],[$0])dnl 127 | AC_CONFIG_COMMANDS([ifelse($1,,$PACKAGE-config.h,$1)],[dnl 128 | AS_VAR_PUSHDEF([_OUT],[ac_prefix_conf_OUT])dnl 129 | AS_VAR_PUSHDEF([_DEF],[ac_prefix_conf_DEF])dnl 130 | AS_VAR_PUSHDEF([_PKG],[ac_prefix_conf_PKG])dnl 131 | AS_VAR_PUSHDEF([_LOW],[ac_prefix_conf_LOW])dnl 132 | AS_VAR_PUSHDEF([_UPP],[ac_prefix_conf_UPP])dnl 133 | AS_VAR_PUSHDEF([_INP],[ac_prefix_conf_INP])dnl 134 | m4_pushdef([_script],[conftest.prefix])dnl 135 | m4_pushdef([_symbol],[m4_cr_Letters[]m4_cr_digits[]_])dnl 136 | _OUT=`echo ifelse($1, , $PACKAGE-config.h, $1)` 137 | _DEF=`echo _$_OUT | sed -e "y:m4_cr_letters:m4_cr_LETTERS[]:" -e "s/@<:@^m4_cr_Letters@:>@/_/g"` 138 | _PKG=`echo ifelse($2, , $PACKAGE, $2)` 139 | _LOW=`echo _$_PKG | sed -e "y:m4_cr_LETTERS-:m4_cr_letters[]_:"` 140 | _UPP=`echo $_PKG | sed -e "y:m4_cr_letters-:m4_cr_LETTERS[]_:" -e "/^@<:@m4_cr_digits@:>@/s/^/_/"` 141 | _INP=`echo "ifelse($3,,,$3)" | sed -e 's/ *//'` 142 | if test ".$_INP" = "."; then 143 | for ac_file in : $CONFIG_HEADERS; do test "_$ac_file" = _: && continue 144 | case "$ac_file" in 145 | *.h) _INP=$ac_file ;; 146 | *) 147 | esac 148 | test ".$_INP" != "." && break 149 | done 150 | fi 151 | if test ".$_INP" = "."; then 152 | case "$_OUT" in 153 | */*) _INP=`basename "$_OUT"` 154 | ;; 155 | *-*) _INP=`echo "$_OUT" | sed -e "s/@<:@_symbol@:>@*-//"` 156 | ;; 157 | *) _INP=config.h 158 | ;; 159 | esac 160 | fi 161 | if test -z "$_PKG" ; then 162 | AC_MSG_ERROR([no prefix for _PREFIX_PKG_CONFIG_H]) 163 | else 164 | if test ! -f "$_INP" ; then if test -f "$srcdir/$_INP" ; then 165 | _INP="$srcdir/$_INP" 166 | fi fi 167 | AC_MSG_NOTICE(creating $_OUT - prefix $_UPP for $_INP defines) 168 | if test -f $_INP ; then 169 | AS_ECHO(["s/^@%:@undef *\\(@<:@m4_cr_LETTERS[]_@:>@\\)/@%:@undef $_UPP""_\\1/"]) > _script 170 | AS_ECHO(["s/^@%:@undef *\\(@<:@m4_cr_letters@:>@\\)/@%:@undef $_LOW""_\\1/"]) >> _script 171 | AS_ECHO(["s/^@%:@def[]ine *\\(@<:@m4_cr_LETTERS[]_@:>@@<:@_symbol@:>@*\\)\\(.*\\)/@%:@ifndef $_UPP""_\\1\\"]) >> _script 172 | AS_ECHO(["@%:@def[]ine $_UPP""_\\1\\2\\"]) >> _script 173 | AS_ECHO(["@%:@endif/"]) >> _script 174 | AS_ECHO(["s/^@%:@def[]ine *\\(@<:@m4_cr_letters@:>@@<:@_symbol@:>@*\\)\\(.*\\)/@%:@ifndef $_LOW""_\\1\\"]) >> _script 175 | AS_ECHO(["@%:@define $_LOW""_\\1\\2\\"]) >> _script 176 | AS_ECHO(["@%:@endif/"]) >> _script 177 | # now executing _script on _DEF input to create _OUT output file 178 | echo "@%:@ifndef $_DEF" >$tmp/pconfig.h 179 | echo "@%:@def[]ine $_DEF 1" >>$tmp/pconfig.h 180 | echo ' ' >>$tmp/pconfig.h 181 | echo /'*' $_OUT. Generated automatically at end of configure. '*'/ >>$tmp/pconfig.h 182 | 183 | sed -f _script $_INP >>$tmp/pconfig.h 184 | echo ' ' >>$tmp/pconfig.h 185 | echo '/* once:' $_DEF '*/' >>$tmp/pconfig.h 186 | echo "@%:@endif" >>$tmp/pconfig.h 187 | if cmp -s $_OUT $tmp/pconfig.h 2>/dev/null; then 188 | AC_MSG_NOTICE([$_OUT is unchanged]) 189 | else 190 | ac_dir=`AS_DIRNAME(["$_OUT"])` 191 | AS_MKDIR_P(["$ac_dir"]) 192 | rm -f "$_OUT" 193 | mv $tmp/pconfig.h "$_OUT" 194 | fi 195 | cp _script _configs.sed 196 | else 197 | AC_MSG_ERROR([input file $_INP does not exist - skip generating $_OUT]) 198 | fi 199 | rm -f conftest.* 200 | fi 201 | m4_popdef([_symbol])dnl 202 | m4_popdef([_script])dnl 203 | AS_VAR_POPDEF([_INP])dnl 204 | AS_VAR_POPDEF([_UPP])dnl 205 | AS_VAR_POPDEF([_LOW])dnl 206 | AS_VAR_POPDEF([_PKG])dnl 207 | AS_VAR_POPDEF([_DEF])dnl 208 | AS_VAR_POPDEF([_OUT])dnl 209 | ],[PACKAGE="$PACKAGE"])]) -------------------------------------------------------------------------------- /hbase/nhc-config.h: -------------------------------------------------------------------------------- 1 | #ifndef _NHC_CONFIG_H 2 | #define _NHC_CONFIG_H 1 3 | 4 | /* nhc-config.h. Generated automatically at end of configure. */ 5 | /* nhc-config.h. Generated from nhc-config.h.in by configure. */ 6 | /* nhc-config.h.in. Generated from configure.ac by autoheader. */ 7 | 8 | /* Define to 1 if you have the header file. */ 9 | #ifndef NHC_HAVE_DLFCN_H 10 | #define NHC_HAVE_DLFCN_H 1 11 | #endif 12 | 13 | /* Define to 1 if you have the header file. */ 14 | #ifndef NHC_HAVE_INTTYPES_H 15 | #define NHC_HAVE_INTTYPES_H 1 16 | #endif 17 | 18 | /* Define to 1 if you have the `folly' library (-lfolly). */ 19 | #ifndef NHC_HAVE_LIBFOLLY 20 | #define NHC_HAVE_LIBFOLLY 1 21 | #endif 22 | 23 | /* Define to 1 if you have the `gflags' library (-lgflags). */ 24 | #ifndef NHC_HAVE_LIBGFLAGS 25 | #define NHC_HAVE_LIBGFLAGS 1 26 | #endif 27 | 28 | /* Define to 1 if you have the `glog' library (-lglog). */ 29 | #ifndef NHC_HAVE_LIBGLOG 30 | #define NHC_HAVE_LIBGLOG 1 31 | #endif 32 | 33 | /* Define to 1 if you have the `gtest' library (-lgtest). */ 34 | #ifndef NHC_HAVE_LIBGTEST 35 | #define NHC_HAVE_LIBGTEST 1 36 | #endif 37 | 38 | /* Define to 1 if you have the `zookeeper_mt' library (-lzookeeper_mt). */ 39 | #ifndef NHC_HAVE_LIBZOOKEEPER_MT 40 | #define NHC_HAVE_LIBZOOKEEPER_MT 1 41 | #endif 42 | 43 | /* Define to 1 if you have the header file. */ 44 | #ifndef NHC_HAVE_MEMORY_H 45 | #define NHC_HAVE_MEMORY_H 1 46 | #endif 47 | 48 | /* Define if g++ supports C++0x features. */ 49 | #ifndef NHC_HAVE_STDCXX_0X 50 | #define NHC_HAVE_STDCXX_0X /**/ 51 | #endif 52 | 53 | /* Define to 1 if you have the header file. */ 54 | #ifndef NHC_HAVE_STDINT_H 55 | #define NHC_HAVE_STDINT_H 1 56 | #endif 57 | 58 | /* Define to 1 if you have the header file. */ 59 | #ifndef NHC_HAVE_STDLIB_H 60 | #define NHC_HAVE_STDLIB_H 1 61 | #endif 62 | 63 | /* Define to 1 if you have the header file. */ 64 | #ifndef NHC_HAVE_STRINGS_H 65 | #define NHC_HAVE_STRINGS_H 1 66 | #endif 67 | 68 | /* Define to 1 if you have the header file. */ 69 | #ifndef NHC_HAVE_STRING_H 70 | #define NHC_HAVE_STRING_H 1 71 | #endif 72 | 73 | /* Define to 1 if you have the header file. */ 74 | #ifndef NHC_HAVE_SYS_STAT_H 75 | #define NHC_HAVE_SYS_STAT_H 1 76 | #endif 77 | 78 | /* Define to 1 if you have the header file. */ 79 | #ifndef NHC_HAVE_SYS_TYPES_H 80 | #define NHC_HAVE_SYS_TYPES_H 1 81 | #endif 82 | 83 | /* Define to 1 if you have the header file. */ 84 | #ifndef NHC_HAVE_UNISTD_H 85 | #define NHC_HAVE_UNISTD_H 1 86 | #endif 87 | 88 | /* Define to the sub-directory in which libtool stores uninstalled libraries. 89 | */ 90 | #ifndef NHC_LT_OBJDIR 91 | #define NHC_LT_OBJDIR ".libs/" 92 | #endif 93 | 94 | /* Always defined for the open source release */ 95 | #ifndef NHC_OPEN_SOURCE 96 | #define NHC_OPEN_SOURCE 0 97 | #endif 98 | 99 | /* Name of package */ 100 | #ifndef NHC_PACKAGE 101 | #define NHC_PACKAGE "nhc" 102 | #endif 103 | 104 | /* Define to the address where bug reports for this package should be sent. */ 105 | #ifndef NHC_PACKAGE_BUGREPORT 106 | #define NHC_PACKAGE_BUGREPORT "folly@fb.com" 107 | #endif 108 | 109 | /* Define to the full name of this package. */ 110 | #ifndef NHC_PACKAGE_NAME 111 | #define NHC_PACKAGE_NAME "nhc" 112 | #endif 113 | 114 | /* Define to the full name and version of this package. */ 115 | #ifndef NHC_PACKAGE_STRING 116 | #define NHC_PACKAGE_STRING "nhc 0.1" 117 | #endif 118 | 119 | /* Define to the one symbol short name of this package. */ 120 | #ifndef NHC_PACKAGE_TARNAME 121 | #define NHC_PACKAGE_TARNAME "nhc" 122 | #endif 123 | 124 | /* Define to the home page for this package. */ 125 | #ifndef NHC_PACKAGE_URL 126 | #define NHC_PACKAGE_URL "" 127 | #endif 128 | 129 | /* Define to the version of this package. */ 130 | #ifndef NHC_PACKAGE_VERSION 131 | #define NHC_PACKAGE_VERSION "0.1" 132 | #endif 133 | 134 | /* Define to 1 if you have the ANSI C header files. */ 135 | #ifndef NHC_STDC_HEADERS 136 | #define NHC_STDC_HEADERS 1 137 | #endif 138 | 139 | /* Version number of package */ 140 | #ifndef NHC_VERSION 141 | #define NHC_VERSION "0.1" 142 | #endif 143 | 144 | /* once: _NHC_CONFIG_H */ 145 | #endif 146 | -------------------------------------------------------------------------------- /hbase/testing/NativeTesting.cpp: -------------------------------------------------------------------------------- 1 | /* Copyright 2012 Facebook, Inc. 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | 16 | // A program that tests hbc functionality against an HBase cluster. 17 | // 18 | // This test is a work in progress. As it is quite difficult to 19 | // instantiate an hbase cell as part of a unit test (from C++, anyway), we 20 | // instead rely on the Eris cluster by default. 21 | // 22 | // However, the preferred way to run this test would be from HBase's 23 | // TestNativeThriftClient unit test. TestNativeThriftClient instantiates a 24 | // Mini HBase cluster and runs this test against that mini-cluster. 25 | // If you are making changes here, you can test them with TestNativeThriftClient 26 | // as follows: 27 | // 28 | // export FBCODE_DIR= 29 | // mvn -Dtest=TestNativeThriftClient test 30 | 31 | #include 32 | #include 33 | #include 34 | 35 | #include "folly/Hash.h" 36 | #include "folly/String.h" 37 | 38 | #include "hbase/NativeHbaseClient.h" 39 | 40 | #include "hbase/hbase_constants.h" 41 | 42 | using namespace facebook; 43 | using namespace std; 44 | using apache::hadoop::hbase::thrift::g_hbase_constants; 45 | namespace h = facebook::hbase; 46 | 47 | DEFINE_string(hbase, "eris002-snc4-hbase-zookeepers", 48 | "SMC tier or zookeeper quorum for the cell to run against"); 49 | DEFINE_string(test_table_name, "hbase_client_test_table", 50 | "Table to create, mangle, and destroy for this test"); 51 | DEFINE_string(permanent_test_table_name, "hbase_client_permanent_test_table", 52 | "Permanent table known to conform to a certain structure, " 53 | "used during testing to ensure proper region boundary behavior"); 54 | DEFINE_int32(num_test_rows, 25000, "number of rows to test with"); 55 | DEFINE_int32(test_batch_size, 1000, "number of rows to batch per change"); 56 | DECLARE_int32(zookeeper_port); 57 | 58 | int main(int argc, char **argv) { 59 | google::InstallFailureSignalHandler(); 60 | google::ParseCommandLineFlags(&argc, &argv, true); 61 | google::InitGoogleLogging(argv[0]); 62 | 63 | if (FLAGS_permanent_test_table_name == FLAGS_test_table_name) { 64 | LOG(FATAL) << "Do not set --test_table_name to be the same as " 65 | << "--permanent_test_table_name!"; 66 | } 67 | 68 | LOG(INFO) << "Connecting to " << FLAGS_hbase; 69 | 70 | h::NativeHbaseClient client(FLAGS_hbase); 71 | if (!client.connect()) { 72 | LOG(FATAL) << "Unable to connect to zookeeper: " << FLAGS_hbase; 73 | } 74 | 75 | // Delete the test table if it already exists 76 | vector tables; 77 | client.listTables(&tables); 78 | for (int i = 0; i < tables.size(); ++i) { 79 | if (tables[i] == FLAGS_test_table_name) { 80 | LOG(INFO) << "Disabling existing table " << tables[i]; 81 | client.disableTable(tables[i]); 82 | LOG(INFO) << "Dropping table " << tables[i]; 83 | client.deleteTable(tables[i]); 84 | break; 85 | } 86 | } 87 | 88 | { 89 | LOG(INFO) << "Creating " << FLAGS_test_table_name; 90 | vector columns; 91 | columns.resize(2); 92 | columns[0].name = "cfam1:"; 93 | columns[1].name = "cfam2:"; 94 | client.createTable(FLAGS_test_table_name, columns); 95 | } 96 | 97 | { 98 | vector columns; 99 | std::unique_ptr scanner(client.getScanner(FLAGS_test_table_name, 100 | "", columns)); 101 | scanner->scan(); 102 | h::TRowResult *result; 103 | if (scanner->next(&result)) { 104 | LOG(FATAL) << "Table should be empty!"; 105 | } 106 | } 107 | 108 | map table_contents; 109 | { 110 | LOG(INFO) << "Inserting rows"; 111 | vector mutations(FLAGS_test_batch_size); 112 | for (uint64_t i = 0; i < FLAGS_num_test_rows; ++i) { 113 | mutations.resize(mutations.size() + 1); 114 | h::BatchMutation &batch = mutations.back(); 115 | batch.row = folly::stringPrintf("%016lx", folly::hash::twang_mix64(i)); 116 | 117 | h::Mutation mutation; 118 | mutation.column = folly::stringPrintf("cfam%ld:col%c", (i & 1) + 1, 119 | 'a' + int(i & 0xf)); 120 | mutation.value = folly::stringPrintf("some value %ld", i); 121 | table_contents[batch.row] = mutation.value; 122 | mutation.isDelete = false; 123 | mutation.writeToWAL = bool(i % 2); 124 | batch.mutations.push_back(mutation); 125 | if (mutations.size() > FLAGS_test_batch_size) { 126 | if (!client.mutateRows(FLAGS_test_table_name, mutations)) { 127 | LOG(FATAL) << "One or more mutation failed"; 128 | } 129 | mutations.clear(); 130 | } 131 | } 132 | if (!client.mutateRows(FLAGS_test_table_name, mutations)) { 133 | LOG(FATAL) << "One or more mutation failed"; 134 | } 135 | } 136 | 137 | { 138 | LOG(INFO) << "Checking unused scanner destruction"; 139 | vector columns; 140 | columns.push_back("cfam1:"); 141 | std::unique_ptr scanner(client.getScanner(FLAGS_test_table_name, 142 | "", columns)); 143 | } 144 | 145 | string third_row_key; 146 | { 147 | LOG(INFO) << "Checking rows"; 148 | vector columns; 149 | columns.push_back("cfam1:"); 150 | std::unique_ptr scanner(client.getScanner(FLAGS_test_table_name, 151 | "", columns)); 152 | scanner->scan(); 153 | h::TRowResult *result; 154 | int num_rows = 0; 155 | while (scanner->next(&result)) { 156 | num_rows++; 157 | if (num_rows == 3) { 158 | third_row_key = result->row; 159 | } 160 | } 161 | if (num_rows != FLAGS_num_test_rows / 2) { 162 | LOG(FATAL) << "Expected " << FLAGS_num_test_rows << " rows, got " 163 | << num_rows; 164 | } 165 | } 166 | 167 | { 168 | CHECK(!third_row_key.empty()); 169 | LOG(INFO) << "Testing end row"; 170 | vector columns; 171 | columns.push_back("cfam1:"); 172 | std::unique_ptr scanner(client.getScanner(FLAGS_test_table_name, 173 | "", columns)); 174 | scanner->setEndRow(third_row_key); 175 | scanner->scan(); 176 | h::TRowResult *result; 177 | int num_rows = 0; 178 | string last_row_key; 179 | while (scanner->next(&result)) { 180 | ++num_rows; 181 | last_row_key = result->row; 182 | CHECK_LE(last_row_key, third_row_key); 183 | } 184 | CHECK_EQ(num_rows, 3); 185 | CHECK_EQ(third_row_key, last_row_key); 186 | } 187 | { 188 | try { 189 | LOG(INFO) << "Querying invalid table"; 190 | vector columns; 191 | std::unique_ptr scanner( 192 | client.getScanner("table_that_should_not_exist", "", columns)); 193 | scanner->scan(); 194 | h::TRowResult *result; 195 | scanner->next(&result); 196 | LOG(FATAL) << "Query should have failed!"; 197 | } catch (const h::TableNotFound& e) { 198 | LOG(INFO) << "Properly could not query bad table"; 199 | } 200 | } 201 | 202 | if (!FLAGS_permanent_test_table_name.empty()) { 203 | const string split_key("6e915c835d215625"); 204 | { 205 | vector regions; 206 | client.listTableRegions(®ions, FLAGS_permanent_test_table_name); 207 | if (regions.size() != 2) { 208 | LOG(FATAL) << FLAGS_permanent_test_table_name << " should have exactly " 209 | << "two regions, not " << regions.size(); 210 | } 211 | if (regions[1].startKey != split_key) { 212 | LOG(FATAL) << "Expected splitpoint of " 213 | << FLAGS_permanent_test_table_name 214 | << "to be " << split_key << ", not " << regions[1].startKey; 215 | } 216 | vector columns; 217 | std::unique_ptr scanner( 218 | client.getScanner(FLAGS_permanent_test_table_name, "", columns)); 219 | 220 | scanner->scan(); 221 | h::TRowResult *result; 222 | int num_rows = 0; 223 | while (scanner->next(&result)) { 224 | num_rows++; 225 | } 226 | if (num_rows != FLAGS_num_test_rows) { 227 | LOG(FATAL) << "Expected " << FLAGS_num_test_rows << " rows, got " 228 | << num_rows; 229 | } 230 | } 231 | 232 | { 233 | LOG(INFO) << "Testing scan with end row that crosses region boundaries"; 234 | vector columns; 235 | std::unique_ptr scanner( 236 | client.getScanner(FLAGS_permanent_test_table_name, "", columns)); 237 | scanner->setEndRow(split_key + "Z"); 238 | scanner->scan(); 239 | 240 | h::TRowResult *result; 241 | string last_row_key; 242 | int num_rows = 0; 243 | while (scanner->next(&result)) { 244 | ++num_rows; 245 | last_row_key = result->row; 246 | } 247 | CHECK_EQ(last_row_key, split_key); 248 | } 249 | } 250 | 251 | { 252 | LOG(INFO) << "Checking single operations"; 253 | vector results; 254 | client.getRow(&results, FLAGS_test_table_name, 255 | "non-existent row"); 256 | CHECK_EQ(0, results.size()); 257 | 258 | vector mutations; 259 | mutations.resize(2); 260 | mutations[0].column = "cfam1:testcol1"; 261 | mutations[0].value = "value 1"; 262 | mutations[1].column = "cfam1:testcol2"; 263 | mutations[1].value = "value 2"; 264 | CHECK(client.mutateRow(FLAGS_test_table_name, "new row", mutations)); 265 | 266 | client.getRow(&results, FLAGS_test_table_name, 267 | "new row"); 268 | CHECK_EQ(1, results.size()); 269 | CHECK_EQ("value 1", results[0].columns.begin()->second.value); 270 | } 271 | 272 | { 273 | LOG(INFO) << "Checking batch row fetch"; 274 | vector results; 275 | vector keys; 276 | for (auto it = table_contents.begin(); it != table_contents.end(); ++it) { 277 | keys.push_back(it->first); 278 | } 279 | keys.push_back("non-existent row key"); 280 | 281 | client.getRow(&results, FLAGS_test_table_name, keys[0]); 282 | CHECK_EQ(results.size(), 1); 283 | CHECK_EQ(table_contents[keys[0]], 284 | results[0].columns.begin()->second.value); 285 | 286 | unordered_map rows_by_key; 287 | CHECK(client.getRows(&results, &rows_by_key, FLAGS_test_table_name, keys)); 288 | CHECK_EQ(results.size(), keys.size() - 1 /* non-existent key */); 289 | for (int i = 0; i < results.size(); ++i) { 290 | CHECK_EQ(table_contents[results[i].row], 291 | results[i].columns.begin()->second.value); 292 | CHECK(rows_by_key[results[i].row] != NULL); 293 | } 294 | CHECK(rows_by_key.find("non-existent row key") != rows_by_key.end()); 295 | CHECK(rows_by_key["non-existent row key"] == NULL); 296 | 297 | LOG(INFO) << "Testing getRowsWithColumns"; 298 | results.clear(); 299 | rows_by_key.clear(); 300 | vector columns { "cfam1:cola", "cfam1:badcolumn" }; 301 | CHECK(client.getRowsWithColumns(&results, &rows_by_key, 302 | FLAGS_test_table_name, keys, columns)); 303 | CHECK_GT(results.size(), 0); 304 | for (int i = 0; i < results.size(); ++i) { 305 | CHECK_EQ(results[i].columns.size(), 1); 306 | CHECK_EQ(results[i].columns.begin()->first, 307 | "cfam1:cola"); 308 | CHECK_EQ(table_contents[results[i].row], 309 | results[i].columns.begin()->second.value); 310 | CHECK(rows_by_key[results[i].row] != NULL); 311 | } 312 | CHECK(rows_by_key.find("non-existent row key") != rows_by_key.end()); 313 | CHECK(rows_by_key["non-existent row key"] == NULL); 314 | } 315 | 316 | { 317 | LOG(INFO) << "Checking per-mutation timestamps"; 318 | vector mutations; 319 | const int64_t custom_ts = 2 * 1000 * 1000 * 1000; 320 | const int64_t default_ts = 1000 * 1000 * 1000; 321 | mutations.resize(2); 322 | 323 | // A mutation with custom timestamp 324 | mutations[0].column = "cfam1:c1"; 325 | mutations[0].value = "v1"; 326 | mutations[0].timestamp = custom_ts; 327 | 328 | // A mutation with default timestamp 329 | mutations[1].column = "cfam1:c2"; 330 | mutations[1].value = "v2"; 331 | CHECK_EQ(g_hbase_constants.LATEST_TIMESTAMP, mutations[1].timestamp); 332 | 333 | const string row("per_mutation_ts"); 334 | CHECK(client.mutateRowTs(FLAGS_test_table_name, row, mutations, 335 | default_ts)); 336 | 337 | vector results; 338 | client.getRow(&results, FLAGS_test_table_name, row); 339 | CHECK_EQ(1, results.size()); 340 | const h::TRowResult &r = results[0]; 341 | CHECK_EQ(row, r.row); 342 | CHECK_EQ(2, r.columns.size()); 343 | for(const auto& m : mutations) { 344 | auto col_iter(r.columns.find(m.column)); 345 | CHECK(col_iter != r.columns.end()); 346 | auto &c(col_iter->second); 347 | CHECK_EQ(m.value, c.value); 348 | const int64_t expected_ts = 349 | m.timestamp == g_hbase_constants.LATEST_TIMESTAMP ? 350 | default_ts : custom_ts; 351 | CHECK_EQ(expected_ts, c.timestamp); 352 | } 353 | } 354 | 355 | LOG(INFO) << "All done!"; 356 | 357 | return 0; 358 | } 359 | --------------------------------------------------------------------------------