You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2019/03/12 12:45:07 UTC

[hbase] 19/133: HBASE-15739 Add region location lookup from meta

This is an automated email from the ASF dual-hosted git repository.

zghao pushed a commit to branch HBASE-14850
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 1bf99b390ec327666b21a0002ff53edc442eabaa
Author: Elliott Clark <ec...@apache.org>
AuthorDate: Sat Apr 30 17:12:11 2016 -0700

    HBASE-15739 Add region location lookup from meta
    
    Summary:
    Add the ability for location-cache to read from meta.
    There's still no cache. Or any parsing.
    
    Test Plan: Needs tests.
    
    Differential Revision: https://reviews.facebook.net/D57489
---
 hbase-native-client/connection/BUCK                |  1 +
 .../connection/client-dispatcher.cc                |  2 +
 hbase-native-client/connection/client-dispatcher.h |  3 +-
 hbase-native-client/connection/client-handler.cc   |  5 +-
 hbase-native-client/connection/client-handler.h    |  2 +-
 .../connection/connection-factory.cc               |  2 +-
 .../connection/connection-factory.h                |  1 +
 hbase-native-client/connection/connection-pool.cc  | 14 +++---
 hbase-native-client/connection/pipeline.cc         |  2 +
 hbase-native-client/connection/pipeline.h          |  1 +
 hbase-native-client/connection/response.h          |  6 ++-
 hbase-native-client/core/BUCK                      |  3 +-
 hbase-native-client/core/location-cache.cc         | 37 +++++++++++++--
 hbase-native-client/core/location-cache.h          | 19 +++++++-
 hbase-native-client/core/meta-utils.cc             | 54 +++++++++++++++++++++-
 hbase-native-client/core/meta-utils.h              |  9 +++-
 .../region-location.h}                             | 26 ++++++-----
 hbase-native-client/core/simple-client.cc          | 48 ++-----------------
 hbase-native-client/core/table-name-test.cc        | 28 ++++++-----
 hbase-native-client/core/table-name.cc             | 34 --------------
 hbase-native-client/core/table-name.h              | 35 +++++++-------
 hbase-native-client/serde/BUCK                     |  1 +
 hbase-native-client/utils/BUCK                     |  7 +--
 hbase-native-client/utils/user-util.cc             |  2 +
 hbase-native-client/utils/user-util.h              |  3 +-
 25 files changed, 201 insertions(+), 144 deletions(-)

diff --git a/hbase-native-client/connection/BUCK b/hbase-native-client/connection/BUCK
index 96f2136..39b8be5 100644
--- a/hbase-native-client/connection/BUCK
+++ b/hbase-native-client/connection/BUCK
@@ -43,6 +43,7 @@ cxx_library(name="connection",
                 "//third-party:folly",
                 "//third-party:wangle",
             ],
+            compiler_flags=['-Weffc++'],
             visibility=['//core/...', ], )
 cxx_test(name="connection-pool-test",
          srcs=["connection-pool-test.cc", ],
diff --git a/hbase-native-client/connection/client-dispatcher.cc b/hbase-native-client/connection/client-dispatcher.cc
index eea0a17..817adc1 100644
--- a/hbase-native-client/connection/client-dispatcher.cc
+++ b/hbase-native-client/connection/client-dispatcher.cc
@@ -22,6 +22,8 @@ using namespace folly;
 using namespace hbase;
 using namespace wangle;
 
+ClientDispatcher::ClientDispatcher() : requests_(), current_call_id_(9) {}
+
 void ClientDispatcher::read(Context *ctx, Response in) {
   auto call_id = in.call_id();
   auto search = requests_.find(call_id);
diff --git a/hbase-native-client/connection/client-dispatcher.h b/hbase-native-client/connection/client-dispatcher.h
index 877e877..826fc6a 100644
--- a/hbase-native-client/connection/client-dispatcher.h
+++ b/hbase-native-client/connection/client-dispatcher.h
@@ -30,6 +30,7 @@ class ClientDispatcher
     : public wangle::ClientDispatcherBase<SerializePipeline,
                                           std::unique_ptr<Request>, Response> {
 public:
+  ClientDispatcher();
   void read(Context *ctx, Response in) override;
   folly::Future<Response> operator()(std::unique_ptr<Request> arg) override;
   folly::Future<folly::Unit> close(Context *ctx) override;
@@ -45,6 +46,6 @@ private:
   //
   // uint32_t has a max of 4Billion so 10 more or less is
   // not a big deal.
-  uint32_t current_call_id_ = 10;
+  uint32_t current_call_id_;
 };
 } // namespace hbase
diff --git a/hbase-native-client/connection/client-handler.cc b/hbase-native-client/connection/client-handler.cc
index abcf5c1..3180f4e 100644
--- a/hbase-native-client/connection/client-handler.cc
+++ b/hbase-native-client/connection/client-handler.cc
@@ -34,8 +34,11 @@ using namespace folly;
 using namespace wangle;
 using hbase::pb::ResponseHeader;
 using hbase::pb::GetResponse;
+using google::protobuf::Message;
 
-ClientHandler::ClientHandler(std::string user_name) : user_name_(user_name) {}
+ClientHandler::ClientHandler(std::string user_name)
+    : user_name_(user_name), need_send_header_(true), ser_(), deser_(),
+      resp_msgs_() {}
 
 void ClientHandler::read(Context *ctx, std::unique_ptr<IOBuf> buf) {
   if (LIKELY(buf != nullptr)) {
diff --git a/hbase-native-client/connection/client-handler.h b/hbase-native-client/connection/client-handler.h
index 41bb883..68513de 100644
--- a/hbase-native-client/connection/client-handler.h
+++ b/hbase-native-client/connection/client-handler.h
@@ -47,7 +47,7 @@ public:
                                    std::unique_ptr<Request> r) override;
 
 private:
-  bool need_send_header_ = true;
+  bool need_send_header_;
   std::string user_name_;
   ClientSerializer ser_;
   ClientDeserializer deser_;
diff --git a/hbase-native-client/connection/connection-factory.cc b/hbase-native-client/connection/connection-factory.cc
index b546269..9102d60 100644
--- a/hbase-native-client/connection/connection-factory.cc
+++ b/hbase-native-client/connection/connection-factory.cc
@@ -40,7 +40,7 @@ using namespace folly;
 using namespace hbase;
 using namespace wangle;
 
-ConnectionFactory::ConnectionFactory() {
+ConnectionFactory::ConnectionFactory() : bootstrap_() {
   bootstrap_.group(std::make_shared<wangle::IOThreadPoolExecutor>(1));
   bootstrap_.pipelineFactory(std::make_shared<RpcPipelineFactory>());
 }
diff --git a/hbase-native-client/connection/connection-factory.h b/hbase-native-client/connection/connection-factory.h
index 5a45316..fc4e161 100644
--- a/hbase-native-client/connection/connection-factory.h
+++ b/hbase-native-client/connection/connection-factory.h
@@ -31,6 +31,7 @@ namespace hbase {
 class ConnectionFactory {
 public:
   ConnectionFactory();
+  virtual ~ConnectionFactory() = default;
 
   virtual std::shared_ptr<HBaseService> make_connection(const std::string &host,
                                                         int port);
diff --git a/hbase-native-client/connection/connection-pool.cc b/hbase-native-client/connection/connection-pool.cc
index a967df2..72c1306 100644
--- a/hbase-native-client/connection/connection-pool.cc
+++ b/hbase-native-client/connection/connection-pool.cc
@@ -35,14 +35,14 @@ class RemoveServiceFilter
 
 public:
   RemoveServiceFilter(std::shared_ptr<HBaseService> service, ServerName sn,
-                      ConnectionPool *cp)
+                      ConnectionPool &cp)
       : ServiceFilter<unique_ptr<Request>, Response>(service), sn_(sn),
         cp_(cp) {}
 
   folly::Future<folly::Unit> close() override {
     if (!released.exchange(true)) {
       return this->service_->close().then(
-          [this]() { this->cp_->close(this->sn_); });
+          [this]() { this->cp_.close(this->sn_); });
     } else {
       return folly::makeFuture();
     }
@@ -57,12 +57,14 @@ public:
 private:
   std::atomic<bool> released{false};
   hbase::pb::ServerName sn_;
-  ConnectionPool *cp_;
+  ConnectionPool &cp_;
 };
 
-ConnectionPool::ConnectionPool() : cf_(std::make_shared<ConnectionFactory>()) {}
+ConnectionPool::ConnectionPool()
+    : cf_(std::make_shared<ConnectionFactory>()), connections_(), map_mutex_() {
+}
 ConnectionPool::ConnectionPool(std::shared_ptr<ConnectionFactory> cf)
-    : cf_(cf) {}
+    : cf_(cf), connections_(), map_mutex_() {}
 
 std::shared_ptr<HBaseService> ConnectionPool::get(const ServerName &sn) {
   SharedMutexWritePriority::UpgradeHolder holder(map_mutex_);
@@ -70,7 +72,7 @@ std::shared_ptr<HBaseService> ConnectionPool::get(const ServerName &sn) {
   if (found == connections_.end() || found->second == nullptr) {
     SharedMutexWritePriority::WriteHolder holder(std::move(holder));
     auto new_con = cf_->make_connection(sn.host_name(), sn.port());
-    auto wrapped = std::make_shared<RemoveServiceFilter>(new_con, sn, this);
+    auto wrapped = std::make_shared<RemoveServiceFilter>(new_con, sn, *this);
     connections_[sn] = wrapped;
     return new_con;
   }
diff --git a/hbase-native-client/connection/pipeline.cc b/hbase-native-client/connection/pipeline.cc
index b9f5e0b..39cce8b 100644
--- a/hbase-native-client/connection/pipeline.cc
+++ b/hbase-native-client/connection/pipeline.cc
@@ -30,6 +30,8 @@ using namespace folly;
 using namespace hbase;
 using namespace wangle;
 
+RpcPipelineFactory::RpcPipelineFactory() : user_util_() {}
+
 SerializePipeline::Ptr
 RpcPipelineFactory::newPipeline(std::shared_ptr<AsyncTransportWrapper> sock) {
   auto pipeline = SerializePipeline::create();
diff --git a/hbase-native-client/connection/pipeline.h b/hbase-native-client/connection/pipeline.h
index 6c4f4ff..e90cb93 100644
--- a/hbase-native-client/connection/pipeline.h
+++ b/hbase-native-client/connection/pipeline.h
@@ -31,6 +31,7 @@ using SerializePipeline =
 
 class RpcPipelineFactory : public wangle::PipelineFactory<SerializePipeline> {
 public:
+  RpcPipelineFactory();
   SerializePipeline::Ptr
   newPipeline(std::shared_ptr<folly::AsyncTransportWrapper> sock) override;
 
diff --git a/hbase-native-client/connection/response.h b/hbase-native-client/connection/response.h
index a7f7939..d2eb19a 100644
--- a/hbase-native-client/connection/response.h
+++ b/hbase-native-client/connection/response.h
@@ -31,10 +31,12 @@ namespace hbase {
 
 class Response {
 public:
-  Response() : call_id_(0) {}
+  Response() : call_id_(0), response_(nullptr) {}
   uint32_t call_id() { return call_id_; }
   void set_call_id(uint32_t call_id) { call_id_ = call_id; }
-  std::shared_ptr<google::protobuf::Message> response() { return response_; }
+  std::shared_ptr<google::protobuf::Message> response() const {
+    return response_;
+  }
   void set_response(std::shared_ptr<google::protobuf::Message> response) {
     response_ = std::move(response);
   }
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index e555ba4..447248b 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -22,6 +22,7 @@ cxx_library(
         "client.h",
         "connection.h",
         "hbase_macros.h",
+        "region-location.h",
         "location-cache.h",
         "table-name.h",
         # TODO: move this out of exported
@@ -32,7 +33,6 @@ cxx_library(
         "client.cc",
         "location-cache.cc",
         "meta-utils.cc",
-        "table-name.cc",
     ],
     deps=[
         "//connection:connection",
@@ -42,6 +42,7 @@ cxx_library(
         "//third-party:wangle",
         "//third-party:zookeeper_mt",
     ],
+    compiler_flags=['-Weffc++'],
     visibility=[
         'PUBLIC',
     ], )
diff --git a/hbase-native-client/core/location-cache.cc b/hbase-native-client/core/location-cache.cc
index c81deba..539051a 100644
--- a/hbase-native-client/core/location-cache.cc
+++ b/hbase-native-client/core/location-cache.cc
@@ -21,20 +21,31 @@
 #include <folly/Logging.h>
 #include <folly/io/IOBuf.h>
 
+#include "connection/response.h"
+#include "if/Client.pb.h"
 #include "if/ZooKeeper.pb.h"
 #include "serde/zk-deserializer.h"
 
 using namespace std;
 using namespace folly;
-using namespace hbase::pb;
-using namespace hbase;
+
+using hbase::Response;
+using hbase::LocationCache;
+using hbase::RegionLocation;
+using hbase::HBaseService;
+using hbase::pb::ScanResponse;
+using hbase::pb::TableName;
+using hbase::pb::ServerName;
+using hbase::pb::MetaRegionServer;
+using hbase::pb::RegionInfo;
 
 // TODO(eclark): make this configurable on client creation
 static const char META_ZNODE_NAME[] = "/hbase/meta-region-server";
 
 LocationCache::LocationCache(string quorum_spec,
                              shared_ptr<folly::Executor> executor)
-    : quorum_spec_(quorum_spec), executor_(executor), meta_promise_(nullptr) {
+    : quorum_spec_(quorum_spec), executor_(executor), meta_promise_(nullptr),
+      meta_lock_(), cp_(), meta_util_() {
   zk_ = zookeeper_init(quorum_spec.c_str(), nullptr, 1000, 0, 0, 0);
 }
 
@@ -89,3 +100,23 @@ ServerName LocationCache::ReadMetaLocation() {
   }
   return mrs.server();
 }
+
+Future<RegionLocation> LocationCache::locateFromMeta(const TableName &tn,
+                                                     const string &row) {
+  return this->LocateMeta()
+      .then([&](ServerName sn) { return this->cp_.get(sn); })
+      .then([&](std::shared_ptr<HBaseService> service) {
+        return (*service)(std::move(meta_util_.make_meta_request(tn, row)));
+      })
+      .then([&](Response resp) {
+        // take the protobuf response and make it into
+        // a region location.
+        return this->parse_response(std::move(resp));
+      });
+}
+
+RegionLocation LocationCache::parse_response(const Response &resp) {
+  auto resp_msg = static_pointer_cast<ScanResponse>(resp.response());
+  LOG(ERROR) << "resp_msg = " << resp_msg->DebugString();
+  return RegionLocation{RegionInfo{}, ServerName{}, nullptr};
+}
diff --git a/hbase-native-client/core/location-cache.h b/hbase-native-client/core/location-cache.h
index ce97c43..cfd6838 100644
--- a/hbase-native-client/core/location-cache.h
+++ b/hbase-native-client/core/location-cache.h
@@ -27,9 +27,19 @@
 #include <mutex>
 #include <string>
 
-#include "if/HBase.pb.h"
+#include "connection/connection-pool.h"
+#include "core/meta-utils.h"
+#include "core/table-name.h"
+#include "core/region-location.h"
 
 namespace hbase {
+
+class Request;
+class Response;
+namespace pb {
+class ServerName;
+}
+
 class LocationCache {
 public:
   explicit LocationCache(std::string quorum_spec,
@@ -38,6 +48,9 @@ public:
   // Meta Related Methods.
   // These are only public until testing is complete
   folly::Future<hbase::pb::ServerName> LocateMeta();
+  folly::Future<RegionLocation> locateFromMeta(const hbase::pb::TableName &tn,
+                                               const std::string &row);
+  RegionLocation parse_response(const Response &resp);
   void InvalidateMeta();
 
 private:
@@ -48,7 +61,11 @@ private:
   std::shared_ptr<folly::Executor> executor_;
   std::unique_ptr<folly::SharedPromise<hbase::pb::ServerName>> meta_promise_;
   std::mutex meta_lock_;
+  ConnectionPool cp_;
+  MetaUtil meta_util_;
+
 
+  // TODO: migrate this to a smart pointer with a deleter.
   zhandle_t *zk_;
 };
 } // namespace hbase
diff --git a/hbase-native-client/core/meta-utils.cc b/hbase-native-client/core/meta-utils.cc
index 7c7d45f..d2fdd88 100644
--- a/hbase-native-client/core/meta-utils.cc
+++ b/hbase-native-client/core/meta-utils.cc
@@ -21,11 +21,61 @@
 
 #include <folly/Conv.h>
 
+#include "connection/request.h"
+#include "connection/response.h"
 #include "core/table-name.h"
+#include "if/Client.pb.h"
 
-using namespace hbase;
+using hbase::pb::TableName;
+using hbase::MetaUtil;
+using hbase::Request;
+using hbase::Response;
+using hbase::pb::ScanRequest;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+
+static const std::string META_REGION = "1588230740";
 
 std::string MetaUtil::region_lookup_rowkey(const TableName &tn,
-                                           const std::string &row) {
+                                           const std::string &row) const {
   return folly::to<std::string>(tn, ",", row, ",", "999999999999999999");
 }
+
+std::unique_ptr<Request>
+MetaUtil::make_meta_request(const TableName tn, const std::string &row) const {
+  auto request = Request::scan();
+  auto msg = std::static_pointer_cast<ScanRequest>(request->req_msg());
+
+  msg->set_number_of_rows(1);
+  msg->set_close_scanner(true);
+
+  // Set the region this scan goes to
+  auto region = msg->mutable_region();
+  region->set_value(META_REGION);
+  region->set_type(RegionSpecifier_RegionSpecifierType::
+                       RegionSpecifier_RegionSpecifierType_ENCODED_REGION_NAME);
+
+  auto scan = msg->mutable_scan();
+  // We don't care about before, just now.
+  scan->set_max_versions(1);
+  // Meta should be cached at all times.
+  scan->set_cache_blocks(true);
+  // We only want one row right now.
+  //
+  // TODO(eclark): Figure out if we should get more.
+  scan->set_caching(1);
+  // Close the scan after we have data.
+  scan->set_small(true);
+  // We know where to start but not where to end.
+  scan->set_reversed(true);
+  // Give me everything or nothing.
+  scan->set_allow_partial_results(false);
+
+  // Set the columns that we need
+  auto info_col = scan->add_column();
+  info_col->set_family("info");
+  info_col->add_qualifier("server");
+  info_col->add_qualifier("regioninfo");
+
+  scan->set_start_row(region_lookup_rowkey(tn, row));
+  return request;
+}
diff --git a/hbase-native-client/core/meta-utils.h b/hbase-native-client/core/meta-utils.h
index 391d0ea..e007d02 100644
--- a/hbase-native-client/core/meta-utils.h
+++ b/hbase-native-client/core/meta-utils.h
@@ -20,11 +20,16 @@
 
 #include <string>
 
+#include "connection/Request.h"
+#include "core/table-name.h"
+
 namespace hbase {
-class TableName;
 
 class MetaUtil {
 public:
-  std::string region_lookup_rowkey(const TableName &tn, const std::string &row);
+  std::string region_lookup_rowkey(const hbase::pb::TableName &tn,
+                                   const std::string &row) const;
+  std::unique_ptr<Request> make_meta_request(const hbase::pb::TableName tn,
+                                             const std::string &row) const;
 };
 } // namespace hbase
diff --git a/hbase-native-client/connection/connection-factory.h b/hbase-native-client/core/region-location.h
similarity index 63%
copy from hbase-native-client/connection/connection-factory.h
copy to hbase-native-client/core/region-location.h
index 5a45316..a46b8e2 100644
--- a/hbase-native-client/connection/connection-factory.h
+++ b/hbase-native-client/core/region-location.h
@@ -18,24 +18,28 @@
  */
 #pragma once
 
-#include <wangle/service/Service.h>
+#include <memory>
 
-#include <string>
-
-#include "connection/pipeline.h"
-#include "connection/request.h"
-#include "connection/response.h"
 #include "connection/service.h"
+#include "if/HBase.pb.h"
+
 
 namespace hbase {
-class ConnectionFactory {
+
+class RegionLocation {
 public:
-  ConnectionFactory();
+  RegionLocation(hbase::pb::RegionInfo ri, hbase::pb::ServerName sn,
+                 std::shared_ptr<HBaseService> service)
+      : ri_(ri), sn_(sn), service_(service) {}
 
-  virtual std::shared_ptr<HBaseService> make_connection(const std::string &host,
-                                                        int port);
+  const hbase::pb::RegionInfo& region_info() { return ri_; }
+  const hbase::pb::ServerName& server_name() { return sn_; }
+  std::shared_ptr<HBaseService> service() { return service_; }
 
 private:
-  wangle::ClientBootstrap<SerializePipeline> bootstrap_;
+  hbase::pb::RegionInfo ri_;
+  hbase::pb::ServerName sn_;
+  std::shared_ptr<HBaseService> service_;
 };
+
 } // namespace hbase
diff --git a/hbase-native-client/core/simple-client.cc b/hbase-native-client/core/simple-client.cc
index 11dcd68..ab614e4 100644
--- a/hbase-native-client/core/simple-client.cc
+++ b/hbase-native-client/core/simple-client.cc
@@ -27,6 +27,7 @@
 
 #include "connection/connection-pool.h"
 #include "core/client.h"
+#include "core/table-name.h"
 #include "if/Client.pb.h"
 #include "if/ZooKeeper.pb.h"
 
@@ -38,13 +39,14 @@ using hbase::Request;
 using hbase::HBaseService;
 using hbase::LocationCache;
 using hbase::ConnectionPool;
+using hbase::TableNameUtil;
 using hbase::pb::ServerName;
 using hbase::pb::RegionSpecifier_RegionSpecifierType;
 using hbase::pb::GetRequest;
 using hbase::pb::GetResponse;
 
 // TODO(eclark): remove the need for this.
-DEFINE_string(region, "1588230740", "What region to send a get to");
+DEFINE_string(table, "t", "What region to send a get");
 DEFINE_string(row, "test", "What row to get");
 DEFINE_string(zookeeper, "localhost:2181", "What zk quorum to talk to");
 
@@ -59,48 +61,8 @@ int main(int argc, char *argv[]) {
   auto cpu_ex = wangle::getCPUExecutor();
   LocationCache cache{FLAGS_zookeeper, cpu_ex};
   auto result =
-      cache.LocateMeta()
-          .then([&cp = cp](ServerName sn) { return cp.get(sn); })
-          .then([](shared_ptr<HBaseService> con) {
-            // Send the request
-            auto r = Request::get();
-            // This is a get request so make that
-            auto req_msg = static_pointer_cast<GetRequest>(r->req_msg());
-            // Set what region
-            req_msg->mutable_region()->set_value(FLAGS_region);
-            // It's always this.
-            req_msg->mutable_region()->set_type(
-                RegionSpecifier_RegionSpecifierType::
-                    RegionSpecifier_RegionSpecifierType_ENCODED_REGION_NAME);
-
-            // What row.
-            req_msg->mutable_get()->set_row(FLAGS_row);
-
-            return (*con)(std::move(r));
-          })
-          .then([](Response resp) {
-            return static_pointer_cast<GetResponse>(resp.response());
-          })
-          .via(cpu_ex.get())
-          .then([](shared_ptr<GetResponse> get_resp) {
-            cout << "GetResponse has_result = " << get_resp->has_result()
-                 << '\n';
-            if (get_resp->has_result()) {
-              auto &r = get_resp->result();
-              cout << "Result cell_size = " << r.cell_size() << endl;
-              for (auto &cell : r.cell()) {
-                cout << "\trow = " << cell.row()
-                     << " family = " << cell.family()
-                     << " qualifier = " << cell.qualifier()
-                     << " timestamp = " << cell.timestamp()
-                     << " value = " << cell.value() << endl;
-              }
-              return 0;
-            }
-
-            return 1;
-          })
+      cache.locateFromMeta(TableNameUtil::create(FLAGS_table), FLAGS_row)
           .get(milliseconds(5000));
 
-  return result;
+  return 0;
 }
diff --git a/hbase-native-client/core/table-name-test.cc b/hbase-native-client/core/table-name-test.cc
index 8aff375..7bad3f1 100644
--- a/hbase-native-client/core/table-name-test.cc
+++ b/hbase-native-client/core/table-name-test.cc
@@ -25,26 +25,30 @@
 #include "core/table-name.h"
 
 using namespace hbase;
+using hbase::pb::TableName;
 
 TEST(TestTableName, TestToStringNoDefault) {
-  TableName tn{"TestTableName"};
+  TableName tn;
+  tn.set_qualifier("TestTableName");
   std::string result = folly::to<std::string>(tn);
   ASSERT_EQ(result.find("default"), std::string::npos);
   ASSERT_EQ("TestTableName", result);
 }
+
+TEST(TestTableName, TestToStringNoDefaltWhenSet) {
+  TableName tn;
+  tn.set_namespace_("default");
+  tn.set_qualifier("TestTableName");
+  std::string result = folly::to<std::string>(tn);
+  ASSERT_EQ(result.find("default"), std::string::npos);
+  ASSERT_EQ("TestTableName", result);
+}
+
 TEST(TestTableName, TestToStringIncludeNS) {
-  TableName tn{"hbase", "acl"};
+  TableName tn;
+  tn.set_namespace_("hbase");
+  tn.set_qualifier("acl");
   std::string result = folly::to<std::string>(tn);
   ASSERT_EQ(result.find("hbase"), 0);
   ASSERT_EQ("hbase:acl", result);
 }
-TEST(TestTableName, TestIsDefault) {
-  TableName default_t1{"in_default"};
-  ASSERT_TRUE(default_t1.is_default_name_space());
-
-  TableName default_t2{"default", "in_also"};
-  ASSERT_TRUE(default_t2.is_default_name_space());
-
-  TableName non_default{"testing", "hmm"};
-  ASSERT_FALSE(non_default.is_default_name_space());
-}
diff --git a/hbase-native-client/core/table-name.cc b/hbase-native-client/core/table-name.cc
deleted file mode 100644
index 33087fd..0000000
--- a/hbase-native-client/core/table-name.cc
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-#include "core/table-name.h"
-
-#include <folly/Conv.h>
-
-using namespace hbase;
-
-TableName::TableName(std::string table_name)
-    : name_space_("default"), table_(table_name) {}
-TableName::TableName(std::string name_space, std::string table_name)
-    : name_space_(name_space), table_(table_name) {}
-bool TableName::is_default_name_space() const {
-  return name_space_.length() == 0 || name_space_ == "default";
-}
-bool TableName::operator==(const TableName &other) const {
-  return name_space_ == other.name_space_ && table_ == other.table_;
-}
diff --git a/hbase-native-client/core/table-name.h b/hbase-native-client/core/table-name.h
index 816f7da..1612667 100644
--- a/hbase-native-client/core/table-name.h
+++ b/hbase-native-client/core/table-name.h
@@ -21,33 +21,30 @@
 #include <memory>
 #include <string>
 
+#include "if/HBase.pb.h"
 #include <folly/Conv.h>
 
 namespace hbase {
-
-// This is the core class of a HBase client.
-class TableName {
-public:
-  explicit TableName(std::string table_name);
-  explicit TableName(std::string name_space, std::string table_name);
-
-  std::string name_space() const { return name_space_; };
-  std::string table() const { return table_; };
-  bool is_default_name_space() const;
-  bool operator==(const TableName &other) const;
-
-private:
-  std::string name_space_;
-  std::string table_;
-};
+namespace pb {
 
 // Provide folly::to<std::string>(TableName);
 template <class String> void toAppend(const TableName &in, String *result) {
-  if (in.is_default_name_space()) {
-    folly::toAppend(in.table(), result);
+  if (!in.has_namespace_() || in.namespace_() == "default") {
+    folly::toAppend(in.qualifier(), result);
   } else {
-    folly::toAppend(in.name_space(), ':', in.table(), result);
+    folly::toAppend(in.namespace_(), ':', in.qualifier(), result);
   }
 }
 
+} // namespace pb
+
+class TableNameUtil {
+public:
+  static ::hbase::pb::TableName create(std::string table_name) {
+    ::hbase::pb::TableName tn;
+    tn.set_namespace_("default");
+    tn.set_qualifier(table_name);
+    return tn;
+  }
+};
 } // namespace hbase
diff --git a/hbase-native-client/serde/BUCK b/hbase-native-client/serde/BUCK
index 0014c0b..539a221 100644
--- a/hbase-native-client/serde/BUCK
+++ b/hbase-native-client/serde/BUCK
@@ -34,6 +34,7 @@ cxx_library(name="serde",
                 ":client-serializer-test",
                 ":client-deserializer-test",
             ],
+            compiler_flags=['-Weffc++'],
             visibility=[
                 'PUBLIC',
             ], )
diff --git a/hbase-native-client/utils/BUCK b/hbase-native-client/utils/BUCK
index ad12f7d..2ab6a89 100644
--- a/hbase-native-client/utils/BUCK
+++ b/hbase-native-client/utils/BUCK
@@ -25,12 +25,13 @@ cxx_library(name="utils",
             deps=[
                 '//third-party:folly',
             ],
+            tests=[
+                ":user-util-test"
+            ],
             visibility=[
                 'PUBLIC',
             ],
-            tests=[
-                ":user-util-test"
-            ], )
+            compiler_flags=['-Weffc++'], )
 cxx_test(name="user-util-test",
          srcs=[
              "user-util-test.cc",
diff --git a/hbase-native-client/utils/user-util.cc b/hbase-native-client/utils/user-util.cc
index b801537..3d963b3 100644
--- a/hbase-native-client/utils/user-util.cc
+++ b/hbase-native-client/utils/user-util.cc
@@ -27,6 +27,8 @@
 using namespace hbase;
 using namespace std;
 
+UserUtil::UserUtil() : init_{false}, user_name_{"drwho"}, m_() {}
+
 string UserUtil::user_name() {
   if (!init_) {
     compute_user_name();
diff --git a/hbase-native-client/utils/user-util.h b/hbase-native-client/utils/user-util.h
index 9ca28e3..da95d78 100644
--- a/hbase-native-client/utils/user-util.h
+++ b/hbase-native-client/utils/user-util.h
@@ -26,11 +26,12 @@
 namespace hbase {
 class UserUtil {
 public:
+  UserUtil();
   std::string user_name();
 
 private:
   void compute_user_name();
-  std::atomic<bool> init_{false};
+  std::atomic<bool> init_;
   std::string user_name_;
   std::mutex m_;
 };