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:01 UTC

[hbase] 13/133: HBASE-15696 Move region location cache serialization into serde

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 3fa6aba493b340cd7affcadd907da5d5085b2710
Author: Elliott Clark <ec...@apache.org>
AuthorDate: Fri Apr 22 22:21:45 2016 -0700

    HBASE-15696 Move region location cache serialization into serde
    
    Summary: Use IOBuf for zk
    
    Test Plan: Added a unit test
    
    Differential Revision: https://reviews.facebook.net/D57147
---
 hbase-native-client/Dockerfile                     |   7 +-
 hbase-native-client/connection/client-dispatcher.h |   8 ++
 hbase-native-client/connection/client-handler.cc   |   7 +-
 hbase-native-client/connection/client-handler.h    |   2 +-
 hbase-native-client/connection/pipeline.h          |   2 +-
 hbase-native-client/core/client.cc                 |   4 +-
 hbase-native-client/core/client.h                  |   4 +-
 hbase-native-client/core/location-cache-test.cc    |   2 +-
 hbase-native-client/core/location-cache.cc         |  48 +++-----
 hbase-native-client/core/location-cache.h          |   4 +-
 hbase-native-client/core/simple-client.cc          |   6 +-
 hbase-native-client/serde/BUCK                     |  10 ++
 .../serde/client-deserializer-test.cc              |   4 +-
 hbase-native-client/serde/client-deserializer.cc   |   4 +-
 .../serde/client-serializer-test.cc                |   2 +-
 hbase-native-client/serde/client-serializer.cc     |   2 +-
 hbase-native-client/serde/client-serializer.h      |   2 +-
 hbase-native-client/serde/zk-deserializer-test.cc  | 123 +++++++++++++++++++++
 hbase-native-client/serde/zk-deserializer.cc       |  78 +++++++++++++
 .../user-util-test.cc => serde/zk-deserializer.h}  |  29 ++---
 hbase-native-client/utils/user-util-test.cc        |   1 +
 21 files changed, 277 insertions(+), 72 deletions(-)

diff --git a/hbase-native-client/Dockerfile b/hbase-native-client/Dockerfile
index 9ccc0d3..1524c56 100644
--- a/hbase-native-client/Dockerfile
+++ b/hbase-native-client/Dockerfile
@@ -19,12 +19,11 @@ FROM pjameson/buck-folly-watchman
 
 ARG CC=/usr/bin/gcc-5
 ARG CXX=/usr/bin/g++-5
-ARG CFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC -g -fno-omit-frame-pointer -O3 -pthread"
-ARG CXXFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC -g -fno-omit-frame-pointer -O3 -pthread"
+ARG CFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC -g -fno-omit-frame-pointer -O2 -pthread"
+ARG CXXFLAGS="-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC -g -fno-omit-frame-pointer -O2 -pthread"
 
-RUN apt-get install -y clang-format-3.7 vim maven inetutils-ping python-pip && \
+RUN apt-get install -y vim maven inetutils-ping python-pip && \
       pip install yapf && \
-      ln -sf /usr/bin/clang-format-3.7 /usr/bin/clang-format && \
       apt-get -qq clean && \
       apt-get -y -qq autoremove && \
       rm -rf /var/lib/{apt,dpkg,cache,log}/ && \
diff --git a/hbase-native-client/connection/client-dispatcher.h b/hbase-native-client/connection/client-dispatcher.h
index c3987c9..89c7119 100644
--- a/hbase-native-client/connection/client-dispatcher.h
+++ b/hbase-native-client/connection/client-dispatcher.h
@@ -37,6 +37,14 @@ public:
 
 private:
   std::unordered_map<uint32_t, folly::Promise<Response>> requests_;
+  // Start at some number way above what could
+  // be there for un-initialized call id counters.
+  //
+  // This makes it easier to make sure that the're are
+  // no access to un-initialized variables.
+  //
+  // uint32_t has a max of 4Billion so 10 more or less is
+  // not a big deal.
   uint32_t current_call_id_ = 10;
 };
 } // namespace hbase
diff --git a/hbase-native-client/connection/client-handler.cc b/hbase-native-client/connection/client-handler.cc
index 74b23ef..7b7794d 100644
--- a/hbase-native-client/connection/client-handler.cc
+++ b/hbase-native-client/connection/client-handler.cc
@@ -19,15 +19,15 @@
 
 #include "connection/client-handler.h"
 
-#include <glog/logging.h>
 #include <folly/Likely.h>
+#include <glog/logging.h>
 
 #include <string>
 
 #include "connection/request.h"
 #include "connection/response.h"
-#include "if/RPC.pb.h"
 #include "if/Client.pb.h"
+#include "if/RPC.pb.h"
 
 using namespace hbase;
 using namespace folly;
@@ -60,6 +60,9 @@ void ClientHandler::read(Context *ctx, std::unique_ptr<IOBuf> buf) {
   }
 }
 
+
+// TODO(eclark): Figure out how to handle the
+// network errors that are going to come.
 Future<Unit> ClientHandler::write(Context *ctx, Request r) {
   // Keep track of if we have sent the header.
   if (UNLIKELY(need_send_header_)) {
diff --git a/hbase-native-client/connection/client-handler.h b/hbase-native-client/connection/client-handler.h
index 38c5725..dbaf5a0 100644
--- a/hbase-native-client/connection/client-handler.h
+++ b/hbase-native-client/connection/client-handler.h
@@ -22,8 +22,8 @@
 
 #include <string>
 
-#include "serde/client-serializer.h"
 #include "serde/client-deserializer.h"
+#include "serde/client-serializer.h"
 
 // Forward decs.
 namespace hbase {
diff --git a/hbase-native-client/connection/pipeline.h b/hbase-native-client/connection/pipeline.h
index 68ade48..8114fab 100644
--- a/hbase-native-client/connection/pipeline.h
+++ b/hbase-native-client/connection/pipeline.h
@@ -18,8 +18,8 @@
  */
 #pragma once
 
-#include <wangle/service/Service.h>
 #include <folly/io/IOBufQueue.h>
+#include <wangle/service/Service.h>
 
 #include "connection/request.h"
 #include "connection/response.h"
diff --git a/hbase-native-client/core/client.cc b/hbase-native-client/core/client.cc
index 893894f..266c239 100644
--- a/hbase-native-client/core/client.cc
+++ b/hbase-native-client/core/client.cc
@@ -21,8 +21,8 @@
 
 #include <folly/Logging.h>
 #include <folly/Random.h>
-#include <glog/logging.h>
 #include <gflags/gflags.h>
+#include <glog/logging.h>
 #include <wangle/concurrent/GlobalExecutor.h>
 
 #include <string>
@@ -36,5 +36,5 @@ using namespace hbase::pb;
 namespace hbase {
 
 Client::Client(string quorum_spec)
-    : location_cache(quorum_spec, wangle::getCPUExecutor()) {}
+    : location_cache_(quorum_spec, wangle::getCPUExecutor()) {}
 }
diff --git a/hbase-native-client/core/client.h b/hbase-native-client/core/client.h
index c2dc226..b583285 100644
--- a/hbase-native-client/core/client.h
+++ b/hbase-native-client/core/client.h
@@ -19,8 +19,8 @@
 
 #pragma once
 
-#include <folly/io/IOBuf.h>
 #include <folly/futures/Future.h>
+#include <folly/io/IOBuf.h>
 
 #include <string>
 
@@ -36,7 +36,7 @@ public:
   folly::Future<GetResult> get(const GetRequest &get_request);
 
 private:
-  LocationCache location_cache;
+  LocationCache location_cache_;
 };
 
 } // namespace hbase
diff --git a/hbase-native-client/core/location-cache-test.cc b/hbase-native-client/core/location-cache-test.cc
index 70ca6f1..f3166fb 100644
--- a/hbase-native-client/core/location-cache-test.cc
+++ b/hbase-native-client/core/location-cache-test.cc
@@ -16,8 +16,8 @@
  * limitations under the License.
  *
  */
-#include <gtest/gtest.h>
 #include <folly/Memory.h>
+#include <gtest/gtest.h>
 #include <wangle/concurrent/GlobalExecutor.h>
 
 #include "location-cache.h"
diff --git a/hbase-native-client/core/location-cache.cc b/hbase-native-client/core/location-cache.cc
index 52e86e3..5925f4a 100644
--- a/hbase-native-client/core/location-cache.cc
+++ b/hbase-native-client/core/location-cache.cc
@@ -16,11 +16,13 @@
  * limitations under the License.
  *
  */
-#include "location-cache.h"
+#include "core/location-cache.h"
 
 #include <folly/Logging.h>
+#include <folly/io/IOBuf.h>
 
 #include "if/ZooKeeper.pb.h"
+#include "serde/zk-deserializer.h"
 
 using namespace std;
 using namespace folly;
@@ -28,7 +30,7 @@ using namespace hbase::pb;
 using namespace hbase;
 
 // TODO(eclark): make this configurable on client creation
-static const char META_LOCATION[] = "/hbase/meta-region-server";
+static const char META_ZNODE_NAME[] = "/hbase/meta-region-server";
 
 LocationCache::LocationCache(string quorum_spec,
                              shared_ptr<folly::Executor> executor)
@@ -66,45 +68,25 @@ void LocationCache::RefreshMetaLocation() {
 }
 
 ServerName LocationCache::ReadMetaLocation() {
-  char contents[4096];
+  auto buf = IOBuf::create(4096);
+  ZkDeserializer derser;
+
   // This needs to be int rather than size_t as that's what ZK expects.
-  int len = sizeof(contents);
+  int len = buf->capacity();
   // TODO(elliott): handle disconnects/reconntion as needed.
-  int zk_result = zoo_get(this->zk_, META_LOCATION, 0, contents, &len, nullptr);
+  int zk_result =
+      zoo_get(this->zk_, META_ZNODE_NAME, 0,
+              reinterpret_cast<char *>(buf->writableData()), &len, nullptr);
+  LOG(ERROR) << "len = " << len;
   if (zk_result != ZOK || len < 9) {
     LOG(ERROR) << "Error getting meta location.";
     throw runtime_error("Error getting meta location");
   }
-  // There should be a magic number for recoverable zk
-  if (static_cast<uint8_t>(contents[0]) != 255) {
-    LOG(ERROR) << "Magic number not in ZK znode data expected 255 got ="
-               << unsigned(static_cast<uint8_t>(contents[0]));
-    throw runtime_error("Magic number not in znode data");
-  }
-  // pos will keep track of skipped bytes.
-  int pos = 1;
-  // How long is the id?
-  int id_len = 0;
-  for (int i = 0; i < 4; ++i) {
-    id_len = id_len << 8;
-    id_len = id_len | static_cast<uint8_t>(contents[pos]);
-    ++pos;
-  }
-  // Skip the id
-  pos += id_len;
-  // Then all protobuf's for HBase are prefixed with a magic string.
-  // PBUF, so we skip that.
-  // TODO(eclark): check to make sure that the magic string is correct
-  // though I am not sure that will get us much.
-  pos += 4;
+  buf->append(len);
 
   MetaRegionServer mrs;
-  // Try to decode the protobuf.
-  // If there's an error bail out.
-  if (mrs.ParseFromArray(contents + pos, len - pos) == false) {
-    LOG(ERROR) << "Error parsing Protobuf Message";
-    throw runtime_error("Error parsing protobuf");
+  if (derser.parse(buf.get(), &mrs) == false) {
+    LOG(ERROR) << "Unable to decode";
   }
-
   return mrs.server();
 }
diff --git a/hbase-native-client/core/location-cache.h b/hbase-native-client/core/location-cache.h
index 28a1ee1..ce97c43 100644
--- a/hbase-native-client/core/location-cache.h
+++ b/hbase-native-client/core/location-cache.h
@@ -18,10 +18,10 @@
  */
 #pragma once
 
-#include <zookeeper/zookeeper.h>
+#include <folly/Executor.h>
 #include <folly/futures/Future.h>
 #include <folly/futures/SharedPromise.h>
-#include <folly/Executor.h>
+#include <zookeeper/zookeeper.h>
 
 #include <memory>
 #include <mutex>
diff --git a/hbase-native-client/core/simple-client.cc b/hbase-native-client/core/simple-client.cc
index 065f70b..5e6f82e 100644
--- a/hbase-native-client/core/simple-client.cc
+++ b/hbase-native-client/core/simple-client.cc
@@ -22,13 +22,13 @@
 #include <gflags/gflags.h>
 #include <wangle/concurrent/GlobalExecutor.h>
 
-#include <iostream>
 #include <chrono>
+#include <iostream>
 
-#include "core/client.h"
 #include "connection/connection-factory.h"
-#include "if/ZooKeeper.pb.h"
+#include "core/client.h"
 #include "if/Client.pb.h"
+#include "if/ZooKeeper.pb.h"
 
 using namespace folly;
 using namespace std;
diff --git a/hbase-native-client/serde/BUCK b/hbase-native-client/serde/BUCK
index 207607f..0014c0b 100644
--- a/hbase-native-client/serde/BUCK
+++ b/hbase-native-client/serde/BUCK
@@ -19,10 +19,12 @@ cxx_library(name="serde",
             exported_headers=[
                 "client-serializer.h",
                 "client-deserializer.h",
+                "zk-deserializer.h",
             ],
             srcs=[
                 "client-serializer.cc",
                 "client-deserializer.cc",
+                "zk-deserializer.cc",
             ],
             deps=[
                 "//if:if",
@@ -52,3 +54,11 @@ cxx_test(name="client-deserializer-test",
              ":serde",
              "//if:if",
          ], )
+cxx_test(name="zk-deserializer-test",
+         srcs=[
+             "zk-deserializer-test.cc",
+         ],
+         deps=[
+             ":serde",
+             "//if:if",
+         ], )
diff --git a/hbase-native-client/serde/client-deserializer-test.cc b/hbase-native-client/serde/client-deserializer-test.cc
index bb57e50..9fef093 100644
--- a/hbase-native-client/serde/client-deserializer-test.cc
+++ b/hbase-native-client/serde/client-deserializer-test.cc
@@ -17,12 +17,12 @@
  *
  */
 
-#include <gtest/gtest.h>
 #include <folly/io/IOBuf.h>
+#include <gtest/gtest.h>
 
+#include "if/Client.pb.h"
 #include "serde/client-deserializer.h"
 #include "serde/client-serializer.h"
-#include "if/Client.pb.h"
 
 using namespace hbase;
 using folly::IOBuf;
diff --git a/hbase-native-client/serde/client-deserializer.cc b/hbase-native-client/serde/client-deserializer.cc
index 118b0d1..acca7ea 100644
--- a/hbase-native-client/serde/client-deserializer.cc
+++ b/hbase-native-client/serde/client-deserializer.cc
@@ -19,10 +19,10 @@
 
 #include "serde/client-deserializer.h"
 
-#include <google/protobuf/message.h>
+#include <folly/Logging.h>
 #include <google/protobuf/io/coded_stream.h>
 #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
-#include <folly/Logging.h>
+#include <google/protobuf/message.h>
 
 using namespace hbase;
 
diff --git a/hbase-native-client/serde/client-serializer-test.cc b/hbase-native-client/serde/client-serializer-test.cc
index b32b55d..9bf38af 100644
--- a/hbase-native-client/serde/client-serializer-test.cc
+++ b/hbase-native-client/serde/client-serializer-test.cc
@@ -22,9 +22,9 @@
 
 #include <string>
 
-#include "serde/client-serializer.h"
 #include "if/HBase.pb.h"
 #include "if/RPC.pb.h"
+#include "serde/client-serializer.h"
 
 using namespace hbase;
 using namespace hbase::pb;
diff --git a/hbase-native-client/serde/client-serializer.cc b/hbase-native-client/serde/client-serializer.cc
index 881b6e4..09b81c8 100644
--- a/hbase-native-client/serde/client-serializer.cc
+++ b/hbase-native-client/serde/client-serializer.cc
@@ -18,8 +18,8 @@
  */
 #include "serde/client-serializer.h"
 
-#include <folly/io/Cursor.h>
 #include <folly/Logging.h>
+#include <folly/io/Cursor.h>
 #include <google/protobuf/io/coded_stream.h>
 #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
 
diff --git a/hbase-native-client/serde/client-serializer.h b/hbase-native-client/serde/client-serializer.h
index 685095d..9c819fe 100644
--- a/hbase-native-client/serde/client-serializer.h
+++ b/hbase-native-client/serde/client-serializer.h
@@ -18,9 +18,9 @@
  */
 #pragma once
 
+#include <cstdint>
 #include <folly/io/IOBuf.h>
 #include <string>
-#include <cstdint>
 
 // Forward
 namespace google {
diff --git a/hbase-native-client/serde/zk-deserializer-test.cc b/hbase-native-client/serde/zk-deserializer-test.cc
new file mode 100644
index 0000000..92d85a0
--- /dev/null
+++ b/hbase-native-client/serde/zk-deserializer-test.cc
@@ -0,0 +1,123 @@
+/*
+ * 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 "serde/zk-deserializer.h"
+
+#include <folly/Logging.h>
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <gtest/gtest.h>
+
+#include "if/ZooKeeper.pb.h"
+
+using namespace hbase;
+using namespace hbase::pb;
+using namespace folly;
+using namespace std;
+using namespace folly::io;
+
+// Test that would test if there's nothing there.
+TEST(TestZkDesializer, TestThrowNoMagicNum) {
+  ZkDeserializer deser;
+  MetaRegionServer mrs;
+
+  auto buf = IOBuf::create(100);
+  buf->append(100);
+  RWPrivateCursor c{buf.get()};
+  c.write<uint8_t>(99);
+  ASSERT_THROW(deser.parse(buf.get(), &mrs), runtime_error);
+}
+
+// Test if the protobuf is in a format that we can't decode
+TEST(TestZkDesializer, TestBadProtoThrow) {
+  ZkDeserializer deser;
+  MetaRegionServer mrs;
+  string magic{"PBUF"};
+
+  // Set ServerName
+  mrs.mutable_server()->set_host_name("test");
+  mrs.mutable_server()->set_port(567);
+  mrs.mutable_server()->set_start_code(9567);
+
+  // One byte magic number
+  // four bytes for id length
+  // four bytes for id
+  // four bytes for PBUF
+  uint32_t start_len = 1 + 4 + 4 + 4;
+  // How large the protobuf will be
+  uint32_t pbuf_size = mrs.ByteSize();
+
+  auto buf = IOBuf::create(start_len + pbuf_size);
+  buf->append(start_len + pbuf_size);
+  RWPrivateCursor c{buf.get()};
+
+  // Write the magic number
+  c.write<uint8_t>(255);
+  // Write the id len
+  c.writeBE<uint32_t>(4);
+  // Write the id
+  c.write<uint32_t>(13);
+  // Write the PBUF string
+  c.push(reinterpret_cast<const uint8_t *>(magic.c_str()), 4);
+
+  // Create the protobuf
+  MetaRegionServer out;
+  ASSERT_THROW(deser.parse(buf.get(), &out), runtime_error);
+}
+
+// Test to make sure the whole thing works.
+TEST(TestZkDesializer, TestNoThrow) {
+  ZkDeserializer deser;
+  MetaRegionServer mrs;
+  string magic{"PBUF"};
+
+  // Set ServerName
+  mrs.mutable_server()->set_host_name("test");
+  mrs.mutable_server()->set_port(567);
+  mrs.mutable_server()->set_start_code(9567);
+
+  // One byte magic number
+  // four bytes for id length
+  // four bytes for id
+  // four bytes for PBUF
+  uint32_t start_len = 1 + 4 + 4 + 4;
+  // How large the protobuf will be
+  uint32_t pbuf_size = mrs.ByteSize();
+
+  auto buf = IOBuf::create(start_len + pbuf_size);
+  buf->append(start_len + pbuf_size);
+  RWPrivateCursor c{buf.get()};
+
+  // Write the magic number
+  c.write<uint8_t>(255);
+  // Write the id len
+  c.writeBE<uint32_t>(4);
+  // Write the id
+  c.write<uint32_t>(13);
+  // Write the PBUF string
+  c.push(reinterpret_cast<const uint8_t *>(magic.c_str()), 4);
+
+  // Now write the serialized protobuf
+  mrs.SerializeWithCachedSizesToArray(buf->writableData() + start_len);
+
+  // Create the protobuf
+  MetaRegionServer out;
+  ASSERT_TRUE(deser.parse(buf.get(), &out));
+  ASSERT_EQ(mrs.server().host_name(), out.server().host_name());
+}
diff --git a/hbase-native-client/serde/zk-deserializer.cc b/hbase-native-client/serde/zk-deserializer.cc
new file mode 100644
index 0000000..33cf809
--- /dev/null
+++ b/hbase-native-client/serde/zk-deserializer.cc
@@ -0,0 +1,78 @@
+/*
+ * 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 "serde/zk-deserializer.h"
+
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <google/protobuf/message.h>
+
+using hbase::ZkDeserializer;
+using std::runtime_error;
+using folly::IOBuf;
+using folly::io::Cursor;
+using google::protobuf::Message;
+
+static const std::string MAGIC_STRING = "PBUF";
+
+bool ZkDeserializer::parse(IOBuf *buf, Message *out) {
+
+  // The format is like this
+  // 1 byte of magic number. 255
+  // 4 bytes of id length.
+  // id_length number of bytes for the id of who put up the znode
+  // 4 bytes of a magic string PBUF
+  // Then the protobuf serialized without a varint header.
+
+  Cursor c{buf};
+
+  // There should be a magic number for recoverable zk
+  uint8_t magic_num = c.read<uint8_t>();
+  if (magic_num != 255) {
+    LOG(ERROR) << "Magic number not in ZK znode data expected 255 got ="
+               << unsigned(magic_num);
+    throw runtime_error("Magic number not in znode data");
+  }
+  // How long is the id?
+  uint32_t id_len = c.readBE<uint32_t>();
+
+  if (id_len >= c.length()) {
+    LOG(ERROR) << "After skiping the if from zookeeper data there's not enough "
+                  "left to read anything else";
+    throw runtime_error("Not enough bytes to decode from zookeeper");
+  }
+
+  // Skip the id
+  c.skip(id_len);
+
+  // Make sure that the magic string is there.
+  if (MAGIC_STRING != c.readFixedString(4)) {
+    LOG(ERROR) << "There was no PBUF magic string.";
+    throw runtime_error("No PBUF magic string in the zookpeeper data.");
+  }
+
+  // Try to decode the protobuf.
+  // If there's an error bail out.
+  if (out->ParseFromArray(c.data(), c.length()) == false) {
+    LOG(ERROR) << "Error parsing Protobuf Message";
+    throw runtime_error("Error parsing protobuf");
+  }
+
+  return true;
+}
diff --git a/hbase-native-client/utils/user-util-test.cc b/hbase-native-client/serde/zk-deserializer.h
similarity index 75%
copy from hbase-native-client/utils/user-util-test.cc
copy to hbase-native-client/serde/zk-deserializer.h
index 2a7434f..aa91661 100644
--- a/hbase-native-client/utils/user-util-test.cc
+++ b/hbase-native-client/serde/zk-deserializer.h
@@ -16,19 +16,20 @@
  * limitations under the License.
  *
  */
+#pragma once
 
-#include <gtest/gtest.h>
-#include <string>
-#include <folly/Logging.h>
-
-#include "utils/user-util.h"
-
-using namespace std;
-using namespace hbase;
-
-TEST(TestUserUtil, TestGetSomething) {
-  UserUtil u_util;
-  string name = u_util.user_name();
-
-  ASSERT_GT(name.length(), 0);
+namespace google {
+namespace protobuf {
+class Message;
 }
+}
+namespace folly {
+class IOBuf;
+}
+
+namespace hbase {
+class ZkDeserializer {
+public:
+  bool parse(folly::IOBuf *buf, google::protobuf::Message *out);
+};
+} // namespace hbase
diff --git a/hbase-native-client/utils/user-util-test.cc b/hbase-native-client/utils/user-util-test.cc
index 2a7434f..1e72bcc 100644
--- a/hbase-native-client/utils/user-util-test.cc
+++ b/hbase-native-client/utils/user-util-test.cc
@@ -30,5 +30,6 @@ TEST(TestUserUtil, TestGetSomething) {
   UserUtil u_util;
   string name = u_util.user_name();
 
+  // TODO shell out to whoami to check this.
   ASSERT_GT(name.length(), 0);
 }