You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2017/09/15 21:20:18 UTC

[01/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14850 95da699b1 -> 128fc3065


http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/bytes-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/bytes-util.cc b/hbase-native-client/src/hbase/utils/bytes-util.cc
new file mode 100644
index 0000000..144b866
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/bytes-util.cc
@@ -0,0 +1,88 @@
+/*
+ * 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 "hbase/utils/bytes-util.h"
+
+#include <bits/stdc++.h>
+#include <boost/predef.h>
+#include <glog/logging.h>
+
+#include <memory>
+#include <string>
+
+namespace hbase {
+
+constexpr char BytesUtil::kHexChars[];
+
+std::string BytesUtil::ToString(int64_t val) {
+  std::string res;
+#if BOOST_ENDIAN_BIG_BYTE || BOOST_ENDIAN_BIG_WORD
+  for (int i = 7; i > 0; i--) {
+    res += (int8_t)(val & 0xffu);
+    val = val >> 8;
+  }
+  res += (int8_t)val;
+#else
+  int64_t mask = 0xff00000000000000u;
+  for (int i = 56; i >= 1; i -= 8) {
+    auto num = ((val & mask) >> i);
+    res += num;
+    mask = mask >> 8;
+  }
+  res += (val & 0xff);
+#endif
+  return res;
+}
+
+int64_t BytesUtil::ToInt64(std::string str) {
+  if (str.length() < 8) {
+    throw std::runtime_error("There are not enough bytes. Expected: 8, actual: " + str.length());
+  }
+  const unsigned char *bytes = reinterpret_cast<unsigned char *>(const_cast<char *>(str.c_str()));
+  int64_t l = 0;
+  for (int i = 0; i < 8; i++) {
+    l <<= 8;
+    l ^= bytes[i];
+  }
+  return l;
+}
+
+std::string BytesUtil::ToStringBinary(const std::string &b, size_t off, size_t len) {
+  std::string result;
+  // Just in case we are passed a 'len' that is > buffer length...
+  if (off >= b.size()) {
+    return result;
+  }
+  if (off + len > b.size()) {
+    len = b.size() - off;
+  }
+  for (size_t i = off; i < off + len; ++i) {
+    int32_t ch = b[i] & 0xFF;
+    if (ch >= ' ' && ch <= '~' && ch != '\\') {
+      result += ch;
+    } else {
+      result += "\\x";
+      result += kHexChars[ch / 0x10];
+      result += kHexChars[ch % 0x10];
+    }
+  }
+  return result;
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/concurrent-map-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/concurrent-map-test.cc b/hbase-native-client/src/hbase/utils/concurrent-map-test.cc
new file mode 100644
index 0000000..b95c442
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/concurrent-map-test.cc
@@ -0,0 +1,36 @@
+/*
+ * 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 <folly/Logging.h>
+#include <gtest/gtest.h>
+#include <string>
+
+#include "hbase/utils/concurrent-map.h"
+
+using hbase::concurrent_map;
+
+TEST(TestConcurrentMap, TestFindAndErase) {
+  concurrent_map<std::string, std::string> map{500};
+
+  map.insert(std::make_pair("foo", "bar"));
+  auto prev = map.find_and_erase("foo");
+  ASSERT_EQ("bar", prev);
+
+  ASSERT_EQ(map.end(), map.find("foo"));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/connection-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/connection-util.cc b/hbase-native-client/src/hbase/utils/connection-util.cc
new file mode 100644
index 0000000..d78507c
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/connection-util.cc
@@ -0,0 +1,26 @@
+/*
+ * 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 "hbase/utils/connection-util.h"
+
+namespace hbase {
+
+const std::vector<uint32_t> ConnectionUtils::kRetryBackoff = {1,   2,   3,   5,   10,  20, 40,
+                                                              100, 100, 100, 100, 200, 200};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/user-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/user-util-test.cc b/hbase-native-client/src/hbase/utils/user-util-test.cc
new file mode 100644
index 0000000..6a022ef
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/user-util-test.cc
@@ -0,0 +1,35 @@
+/*
+ * 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 <folly/Logging.h>
+#include <gtest/gtest.h>
+#include <string>
+
+#include "hbase/utils/user-util.h"
+
+using namespace std;
+using namespace hbase;
+
+TEST(TestUserUtil, TestGetSomething) {
+  UserUtil u_util;
+  string name = u_util.user_name(false);
+
+  // TODO shell out to whoami to check this.
+  ASSERT_GT(name.length(), 0);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/user-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/user-util.cc b/hbase-native-client/src/hbase/utils/user-util.cc
new file mode 100644
index 0000000..ec8bb27
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/user-util.cc
@@ -0,0 +1,77 @@
+/*
+ * 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 "hbase/utils/user-util.h"
+
+#include <folly/Logging.h>
+#include <krb5/krb5.h>
+#include <pwd.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+namespace hbase {
+
+UserUtil::UserUtil() : once_flag_{} {}
+
+std::string UserUtil::user_name(bool secure) {
+  std::call_once(once_flag_, [this, secure]() { compute_user_name(secure); });
+  return user_name_;
+}
+
+void UserUtil::compute_user_name(bool secure) {
+  // According to the man page of getpwuid
+  // this should never be free'd
+  //
+  // So yeah a raw pointer with no ownership....
+  struct passwd *passwd = getpwuid(getuid());
+
+  // make sure that we got something.
+  if (passwd && passwd->pw_name) {
+    user_name_ = std::string{passwd->pw_name};
+  }
+  if (!secure) return;
+  krb5_context ctx;
+  krb5_error_code ret = krb5_init_context(&ctx);
+  if (ret != 0) {
+    throw std::runtime_error("cannot init krb ctx " + std::to_string(ret));
+  }
+  krb5_ccache ccache;
+  ret = krb5_cc_default(ctx, &ccache);
+  if (ret != 0) {
+    throw std::runtime_error("cannot get default cache " + std::to_string(ret));
+  }
+  // Here is sample principal: hbase/23a03935850c@EXAMPLE.COM
+  // There may be one (user) or two (user/host) components before the @ sign
+  krb5_principal princ;
+  ret = krb5_cc_get_principal(ctx, ccache, &princ);
+  if (ret != 0) {
+    throw std::runtime_error("cannot get default principal " + std::to_string(ret));
+  }
+  user_name_ = princ->data->data;
+  if (krb5_princ_size(ctx, princ) >= 2) {
+    user_name_ += "/";
+    user_name_ += static_cast<char *>(princ->data[1].data);
+  }
+  user_name_ += "@";
+  user_name_ += princ->realm.data;
+  VLOG(1) << "user " << user_name_;
+  krb5_free_principal(ctx, princ);
+  krb5_free_context(ctx);
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/test-util/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/test-util/BUCK b/hbase-native-client/test-util/BUCK
deleted file mode 100644
index 7c92841..0000000
--- a/hbase-native-client/test-util/BUCK
+++ /dev/null
@@ -1,52 +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.
-import os
-
-cxx_library(
-    name="test-util",
-    exported_headers=["test-util.h", "mini-cluster.h"],
-    srcs=["test-util.cc", "mini-cluster.cc"],
-    deps=[
-        "//third-party:folly",
-        "//core:core",
-    ],
-    preprocessor_flags=[
-        '-I' + os.environ['JAVA_HOME'] + '/include',
-        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
-        '-I' + os.environ['JAVA_HOME'] + '/include/linux'
-    ],
-    exported_preprocessor_flags=[
-        '-I' + os.environ['JAVA_HOME'] + '/include',
-        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
-        '-I' + os.environ['JAVA_HOME'] + '/include/linux'
-    ],
-    compiler_flags=[
-        '-I' + os.environ['JAVA_HOME'] + '/include',
-        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
-        '-I' + os.environ['JAVA_HOME'] + '/include/linux', '-ggdb'
-    ],
-    linker_flags=[
-        '-ljvm', '-L' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server',
-        '-ggdb'
-    ],
-    exported_linker_flags=[
-        '-ljvm', '-L' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server',
-        '-Wl,-rpath=' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server'
-    ],
-    visibility=[
-        'PUBLIC',
-    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/test-util/mini-cluster.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/test-util/mini-cluster.cc b/hbase-native-client/test-util/mini-cluster.cc
deleted file mode 100644
index 9dd2f12..0000000
--- a/hbase-native-client/test-util/mini-cluster.cc
+++ /dev/null
@@ -1,311 +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 "test-util/mini-cluster.h"
-#include <fcntl.h>
-#include <glog/logging.h>
-#include <boost/filesystem/fstream.hpp>
-#include <boost/filesystem/operations.hpp>
-#include <fstream>
-
-using hbase::MiniCluster;
-
-JNIEnv *MiniCluster::CreateVM(JavaVM **jvm) {
-  JavaVMInitArgs args;
-  JavaVMOption jvm_options;
-  args.version = JNI_VERSION_1_6;
-  args.nOptions = 1;
-  char *classpath = getenv("CLASSPATH");
-  std::string clspath;
-  if (classpath == NULL || strstr(classpath, "-tests.jar") == NULL) {
-    std::string clsPathFilePath("../target/cached_classpath.txt");
-    std::ifstream fd(clsPathFilePath);
-    std::string prefix("");
-    if (fd.is_open()) {
-      if (classpath == NULL) {
-        LOG(INFO) << "got empty classpath";
-      } else {
-        // prefix bootstrapper.jar
-        prefix.assign(classpath);
-      }
-      std::string line;
-      if (getline(fd, line)) {
-        clspath = prefix + ":" + line;
-        int ret = setenv("CLASSPATH", clspath.c_str(), 1);
-        LOG(INFO) << "set clspath " << ret;
-      } else {
-        LOG(INFO) << "nothing read from " << clsPathFilePath;
-        exit(-1);
-      }
-    } else {
-      LOG(INFO) << "nothing read from " << clsPathFilePath;
-      exit(-1);
-    }
-    fd.close();
-  }
-
-  auto options = std::string{"-Djava.class.path="} + clspath;
-  jvm_options.optionString = const_cast<char *>(options.c_str());
-  args.options = &jvm_options;
-  args.ignoreUnrecognized = 0;
-  int rv;
-  rv = JNI_CreateJavaVM(jvm, reinterpret_cast<void **>(&env_), &args);
-  CHECK(rv >= 0 && env_);
-  return env_;
-}
-
-MiniCluster::~MiniCluster() {
-  if (jvm_ != NULL) {
-    jvm_->DestroyJavaVM();
-    jvm_ = NULL;
-  }
-  env_ = nullptr;
-}
-
-void MiniCluster::Setup() {
-  jmethodID constructor;
-  pthread_mutex_lock(&count_mutex_);
-  if (env_ == NULL) {
-    env_ = CreateVM(&jvm_);
-    if (env_ == NULL) {
-      exit(-1);
-    }
-    testing_util_class_ = env_->FindClass("org/apache/hadoop/hbase/HBaseTestingUtility");
-    // this should be converted to a globalref I think to avoid the underlying java obj getting
-    // GC'ed
-    if (testing_util_class_ == NULL) {
-      LOG(INFO) << "Couldn't find class HBaseTestingUtility";
-      exit(-1);
-    }
-    jmethodID mid = env_->GetStaticMethodID(testing_util_class_, "createLocalHTU",
-                                            "()Lorg/apache/hadoop/hbase/HBaseTestingUtility;");
-    htu_ = env_->CallStaticObjectMethod(testing_util_class_, mid);
-    // this should be converted to a globalref I think to avoid the underlying java obj getting
-    // GC'ed
-    if (htu_ == NULL) {
-      LOG(INFO) << "Couldn't invoke method createLocalHTU in HBaseTestingUtility";
-      exit(-1);
-    }
-    get_conn_mid_ = env_->GetMethodID(testing_util_class_, "getConnection",
-                                      "()Lorg/apache/hadoop/hbase/client/Connection;");
-    jclass conn_class = env_->FindClass("org/apache/hadoop/hbase/client/Connection");
-    get_admin_mid_ =
-        env_->GetMethodID(conn_class, "getAdmin", "()Lorg/apache/hadoop/hbase/client/Admin;");
-    get_table_mid_ = env_->GetMethodID(
-        conn_class, "getTable",
-        "(Lorg/apache/hadoop/hbase/TableName;)Lorg/apache/hadoop/hbase/client/Table;");
-    if (get_table_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find getConnection";
-      exit(-1);
-    }
-    jclass adminClass = env_->FindClass("org/apache/hadoop/hbase/client/Admin");
-    move_mid_ = env_->GetMethodID(adminClass, "move", "([B[B)V");
-    if (move_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find move";
-      exit(-1);
-    }
-    create_table_mid_ =
-        env_->GetMethodID(testing_util_class_, "createTable",
-                          "(Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;)Lorg/"
-                          "apache/hadoop/hbase/client/Table;");
-    create_table_families_mid_ = env_->GetMethodID(testing_util_class_, "createTable",
-                                                   "(Lorg/apache/hadoop/hbase/TableName;[[B)Lorg/"
-                                                   "apache/hadoop/hbase/client/Table;");
-    create_table_with_split_mid_ = env_->GetMethodID(
-        testing_util_class_, "createTable",
-        "(Lorg/apache/hadoop/hbase/TableName;[[B[[B)Lorg/apache/hadoop/hbase/client/Table;");
-    if (create_table_with_split_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find method createTable with split";
-      exit(-1);
-    }
-
-    table_name_class_ = env_->FindClass("org/apache/hadoop/hbase/TableName");
-    tbl_name_value_of_mid_ = env_->GetStaticMethodID(
-        table_name_class_, "valueOf", "(Ljava/lang/String;)Lorg/apache/hadoop/hbase/TableName;");
-    if (tbl_name_value_of_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find method valueOf in TableName";
-      exit(-1);
-    }
-    jclass hbaseMiniClusterClass = env_->FindClass("org/apache/hadoop/hbase/MiniHBaseCluster");
-    stop_rs_mid_ =
-        env_->GetMethodID(hbaseMiniClusterClass, "stopRegionServer",
-                          "(I)Lorg/apache/hadoop/hbase/util/JVMClusterUtil$RegionServerThread;");
-    get_conf_mid_ = env_->GetMethodID(hbaseMiniClusterClass, "getConfiguration",
-                                      "()Lorg/apache/hadoop/conf/Configuration;");
-
-    conf_class_ = env_->FindClass("org/apache/hadoop/conf/Configuration");
-    set_conf_mid_ =
-        env_->GetMethodID(conf_class_, "set", "(Ljava/lang/String;Ljava/lang/String;)V");
-    if (set_conf_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find method getConf in MiniHBaseCluster";
-      exit(-1);
-    }
-    conf_get_mid_ = env_->GetMethodID(conf_class_, "get", "(Ljava/lang/String;)Ljava/lang/String;");
-
-    jclass tableClass = env_->FindClass("org/apache/hadoop/hbase/client/Table");
-    put_mid_ = env_->GetMethodID(tableClass, "put", "(Lorg/apache/hadoop/hbase/client/Put;)V");
-    jclass connFactoryClass = env_->FindClass("org/apache/hadoop/hbase/client/ConnectionFactory");
-    create_conn_mid_ = env_->GetStaticMethodID(connFactoryClass, "createConnection",
-                                               "()Lorg/apache/hadoop/hbase/client/Connection;");
-    if (create_conn_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find createConnection";
-      exit(-1);
-    }
-    put_class_ = env_->FindClass("org/apache/hadoop/hbase/client/Put");
-    put_ctor_ = env_->GetMethodID(put_class_, "<init>", "([B)V");
-    add_col_mid_ =
-        env_->GetMethodID(put_class_, "addColumn", "([B[B[B)Lorg/apache/hadoop/hbase/client/Put;");
-    if (add_col_mid_ == NULL) {
-      LOG(INFO) << "Couldn't find method addColumn";
-      exit(-1);
-    }
-  }
-  pthread_mutex_unlock(&count_mutex_);
-}
-
-jobject MiniCluster::htu() {
-  Setup();
-  return htu_;
-}
-
-JNIEnv *MiniCluster::env() {
-  Setup();
-  return env_;
-}
-// converts C char* to Java byte[]
-jbyteArray MiniCluster::StrToByteChar(const std::string &str) {
-  if (str.length() == 0) {
-    return nullptr;
-  }
-  int n = str.length();
-  jbyteArray arr = env_->NewByteArray(n);
-  env_->SetByteArrayRegion(arr, 0, n, reinterpret_cast<const jbyte *>(str.c_str()));
-  return arr;
-}
-
-jobject MiniCluster::CreateTable(const std::string &table, const std::string &family) {
-  jstring table_name_str = env_->NewStringUTF(table.c_str());
-  jobject table_name =
-      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
-  jstring family_str = env_->NewStringUTF(family.c_str());
-  jobject table_obj = env_->CallObjectMethod(htu_, create_table_mid_, table_name, family_str);
-  return table_obj;
-}
-
-jobject MiniCluster::CreateTable(const std::string &table,
-                                 const std::vector<std::string> &families) {
-  jstring table_name_str = env_->NewStringUTF(table.c_str());
-  jobject table_name =
-      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
-  jclass array_element_type = env_->FindClass("[B");
-  jobjectArray family_array = env_->NewObjectArray(families.size(), array_element_type, nullptr);
-  int i = 0;
-  for (auto family : families) {
-    env_->SetObjectArrayElement(family_array, i++, StrToByteChar(family));
-  }
-  jobject table_obj =
-      env_->CallObjectMethod(htu_, create_table_families_mid_, table_name, family_array);
-  return table_obj;
-}
-
-jobject MiniCluster::CreateTable(const std::string &table, const std::string &family,
-                                 const std::vector<std::string> &keys) {
-  std::vector<std::string> families{};
-  families.push_back(std::string{family});
-  return CreateTable(table, families, keys);
-}
-
-jobject MiniCluster::CreateTable(const std::string &table, const std::vector<std::string> &families,
-                                 const std::vector<std::string> &keys) {
-  jstring table_name_str = env_->NewStringUTF(table.c_str());
-  jobject table_name =
-      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
-  jclass array_element_type = env_->FindClass("[B");
-
-  int i = 0;
-  jobjectArray family_array = env_->NewObjectArray(families.size(), array_element_type, nullptr);
-  for (auto family : families) {
-    env_->SetObjectArrayElement(family_array, i++, StrToByteChar(family));
-  }
-
-  jobjectArray key_array = env_->NewObjectArray(keys.size(), array_element_type, nullptr);
-
-  i = 0;
-  for (auto key : keys) {
-    env_->SetObjectArrayElement(key_array, i++, StrToByteChar(key));
-  }
-
-  jobject tbl = env_->CallObjectMethod(htu_, create_table_with_split_mid_, table_name, family_array,
-                                       key_array);
-  return tbl;
-}
-
-jobject MiniCluster::StopRegionServer(int idx) {
-  env();
-  return env_->CallObjectMethod(cluster_, stop_rs_mid_, (jint)idx);
-}
-
-// returns the Configuration for the cluster
-jobject MiniCluster::GetConf() {
-  env();
-  return env_->CallObjectMethod(cluster_, get_conf_mid_);
-}
-// return the Admin instance for the local cluster
-jobject MiniCluster::admin() {
-  env();
-  jobject conn = env_->CallObjectMethod(htu(), get_conn_mid_);
-  jobject admin = env_->CallObjectMethod(conn, get_admin_mid_);
-  return admin;
-}
-
-// moves region to server
-void MiniCluster::MoveRegion(const std::string &region, const std::string &server) {
-  jobject admin_ = admin();
-  env_->CallObjectMethod(admin_, move_mid_, StrToByteChar(region), StrToByteChar(server));
-}
-
-jobject MiniCluster::StartCluster(int num_region_servers) {
-  env();
-  jmethodID mid = env_->GetMethodID(testing_util_class_, "startMiniCluster",
-                                    "(I)Lorg/apache/hadoop/hbase/MiniHBaseCluster;");
-  if (mid == NULL) {
-    LOG(INFO) << "Couldn't find method startMiniCluster in the class HBaseTestingUtility";
-    exit(-1);
-  }
-  cluster_ = env_->CallObjectMethod(htu(), mid, static_cast<jint>(num_region_servers));
-  return cluster_;
-}
-
-void MiniCluster::StopCluster() {
-  env();
-  jmethodID mid = env_->GetMethodID(testing_util_class_, "shutdownMiniCluster", "()V");
-  env_->CallVoidMethod(htu(), mid);
-  if (jvm_ != NULL) {
-    jvm_->DestroyJavaVM();
-    jvm_ = NULL;
-  }
-}
-
-const std::string MiniCluster::GetConfValue(const std::string &key) {
-  jobject conf = GetConf();
-  jstring jval =
-      (jstring)env_->CallObjectMethod(conf, conf_get_mid_, env_->NewStringUTF(key.c_str()));
-  const char *val = env_->GetStringUTFChars(jval, 0);
-  return val;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/test-util/mini-cluster.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/test-util/mini-cluster.h b/hbase-native-client/test-util/mini-cluster.h
deleted file mode 100644
index 6b4547c..0000000
--- a/hbase-native-client/test-util/mini-cluster.h
+++ /dev/null
@@ -1,81 +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.
- *
- */
-#pragma once
-
-#include <jni.h>
-#include <string>
-#include <vector>
-
-namespace hbase {
-
-class MiniCluster {
- public:
-  virtual ~MiniCluster();
-  jobject StartCluster(int32_t num_region_servers);
-  void StopCluster();
-  jobject CreateTable(const std::string &table, const std::string &family);
-  jobject CreateTable(const std::string &table, const std::vector<std::string> &families);
-  jobject CreateTable(const std::string &table, const std::string &family,
-                      const std::vector<std::string> &keys);
-  jobject CreateTable(const std::string &table, const std::vector<std::string> &families,
-                      const std::vector<std::string> &keys);
-  jobject StopRegionServer(int idx);
-
-  // moves region to server
-  void MoveRegion(const std::string &region, const std::string &server);
-  // returns the Configuration instance for the cluster
-  jobject GetConf();
-  // returns the value for config key retrieved from cluster
-  const std::string GetConfValue(const std::string &key);
-
- private:
-  JNIEnv *env_;
-  jclass testing_util_class_;
-  jclass table_name_class_;
-  jclass put_class_;
-  jclass conf_class_;
-  jmethodID stop_rs_mid_;
-  jmethodID get_conf_mid_;
-  jmethodID set_conf_mid_;
-  jmethodID tbl_name_value_of_mid_;
-  jmethodID create_table_mid_;
-  jmethodID create_table_families_mid_;
-  jmethodID create_table_with_split_mid_;
-  jmethodID put_mid_;
-  jmethodID put_ctor_;
-  jmethodID add_col_mid_;
-  jmethodID create_conn_mid_;
-  jmethodID get_conn_mid_;
-  jmethodID get_table_mid_;
-  jmethodID conf_get_mid_;
-  jmethodID get_admin_mid_;
-  jmethodID move_mid_;
-  jmethodID str_ctor_mid_;
-  jobject htu_;
-  jobject cluster_;
-  pthread_mutex_t count_mutex_;
-  JavaVM *jvm_;
-  JNIEnv *CreateVM(JavaVM **jvm);
-  void Setup();
-  jobject htu();
-  JNIEnv *env();
-  jbyteArray StrToByteChar(const std::string &str);
-  jobject admin();
-};
-} /*namespace hbase*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/test-util/test-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/test-util/test-util.cc b/hbase-native-client/test-util/test-util.cc
deleted file mode 100644
index ea18b84..0000000
--- a/hbase-native-client/test-util/test-util.cc
+++ /dev/null
@@ -1,105 +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 "test-util/test-util.h"
-#include <string.h>
-
-#include <folly/Format.h>
-
-#include "core/zk-util.h"
-
-using hbase::TestUtil;
-using folly::Random;
-
-std::string TestUtil::RandString(int len) {
-  // Create the whole string.
-  // Filling everything with z's
-  auto s = std::string(len, 'z');
-
-  // Now pick a bunch of random numbers
-  for (int i = 0; i < len; i++) {
-    // use Folly's random to get the numbers
-    // as I don't want to have to learn
-    // all the cpp rand invocation magic.
-    auto r = Random::rand32('a', 'z');
-    // Cast that to ascii.
-    s[i] = static_cast<char>(r);
-  }
-  return s;
-}
-
-TestUtil::TestUtil() : temp_dir_(TestUtil::RandString()) {}
-
-TestUtil::~TestUtil() {
-  if (mini_) {
-    StopMiniCluster();
-    mini_ = nullptr;
-  }
-}
-
-void TestUtil::StartMiniCluster(int32_t num_region_servers) {
-  mini_ = std::make_unique<MiniCluster>();
-  mini_->StartCluster(num_region_servers);
-
-  conf()->Set(ZKUtil::kHBaseZookeeperQuorum_, mini_->GetConfValue(ZKUtil::kHBaseZookeeperQuorum_));
-  conf()->Set(ZKUtil::kHBaseZookeeperClientPort_,
-              mini_->GetConfValue(ZKUtil::kHBaseZookeeperClientPort_));
-}
-
-void TestUtil::StopMiniCluster() { mini_->StopCluster(); }
-
-void TestUtil::CreateTable(const std::string &table, const std::string &family) {
-  mini_->CreateTable(table, family);
-}
-
-void TestUtil::CreateTable(const std::string &table, const std::vector<std::string> &families) {
-  mini_->CreateTable(table, families);
-}
-
-void TestUtil::CreateTable(const std::string &table, const std::string &family,
-                           const std::vector<std::string> &keys) {
-  mini_->CreateTable(table, family, keys);
-}
-
-void TestUtil::CreateTable(const std::string &table, const std::vector<std::string> &families,
-                           const std::vector<std::string> &keys) {
-  mini_->CreateTable(table, families, keys);
-}
-
-void TestUtil::MoveRegion(const std::string &region, const std::string &server) {
-  mini_->MoveRegion(region, server);
-}
-
-void TestUtil::StartStandAloneInstance() {
-  auto p = temp_dir_.path().string();
-  auto cmd = std::string{"bin/start-local-hbase.sh " + p};
-  auto res_code = std::system(cmd.c_str());
-  CHECK_EQ(res_code, 0);
-}
-
-void TestUtil::StopStandAloneInstance() {
-  auto res_code = std::system("bin/stop-local-hbase.sh");
-  CHECK_EQ(res_code, 0);
-}
-
-void TestUtil::RunShellCmd(const std::string &command) {
-  auto cmd_string = folly::sformat("echo \"{}\" | ../bin/hbase shell", command);
-  auto res_code = std::system(cmd_string.c_str());
-  CHECK_EQ(res_code, 0);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/test-util/test-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/test-util/test-util.h b/hbase-native-client/test-util/test-util.h
deleted file mode 100644
index 40e99d1..0000000
--- a/hbase-native-client/test-util/test-util.h
+++ /dev/null
@@ -1,78 +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.
- *
- */
-#pragma once
-
-#include <folly/Random.h>
-#include <folly/experimental/TestUtil.h>
-
-#include <cstdlib>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/configuration.h"
-#include "test-util/mini-cluster.h"
-
-namespace hbase {
-/**
- * @brief Class to deal with a local instance cluster for testing.
- */
-class TestUtil {
- public:
-  TestUtil();
-
-  /**
-   * Destroying a TestUtil will spin down a cluster and remove the test dir.
-   */
-  ~TestUtil();
-
-  /**
-   * Create a random string. This random string is all letters, as such it is
-   * very good for use as a directory name.
-   */
-  static std::string RandString(int len = 32);
-
-  /**
-   * Returns the configuration to talk to the local cluster
-   */
-  std::shared_ptr<Configuration> conf() const { return conf_; }
-
-  /**
-   * Starts mini hbase cluster with specified number of region servers
-   */
-  void StartMiniCluster(int32_t num_region_servers);
-
-  void StopMiniCluster();
-  void CreateTable(const std::string &table, const std::string &family);
-  void CreateTable(const std::string &table, const std::vector<std::string> &families);
-  void CreateTable(const std::string &table, const std::string &family,
-                   const std::vector<std::string> &keys);
-  void CreateTable(const std::string &table, const std::vector<std::string> &families,
-                   const std::vector<std::string> &keys);
-
-  void StartStandAloneInstance();
-  void StopStandAloneInstance();
-  void RunShellCmd(const std::string &);
-  void MoveRegion(const std::string &region, const std::string &server);
-
- private:
-  std::unique_ptr<MiniCluster> mini_;
-  folly::test::TemporaryDirectory temp_dir_;
-  std::shared_ptr<Configuration> conf_ = std::make_shared<Configuration>();
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/BUCK b/hbase-native-client/utils/BUCK
deleted file mode 100644
index 96f02b8..0000000
--- a/hbase-native-client/utils/BUCK
+++ /dev/null
@@ -1,64 +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.
-
-cxx_library(
-    name="utils",
-    exported_headers=[
-        "bytes-util.h",
-        "connection-util.h",
-        "concurrent-map.h",
-        "optional.h",
-        "sys-util.h",
-        "time-util.h",
-        "user-util.h",
-        "version.h",
-    ],
-    srcs=["bytes-util.cc", "connection-util.cc", "user-util.cc"],
-    deps=[
-        '//third-party:folly',
-    ],
-    tests=[":user-util-test"],
-    linker_flags=['-L/usr/local/lib', '-lkrb5'],
-    exported_linker_flags=['-L/usr/local/lib', '-lkrb5'],
-    visibility=[
-        'PUBLIC',
-    ],
-    compiler_flags=['-Weffc++'],)
-cxx_test(
-    name="bytes-util-test",
-    srcs=[
-        "bytes-util-test.cc",
-    ],
-    deps=[
-        ":utils",
-    ],)
-cxx_test(
-    name="concurrent-map-test",
-    srcs=[
-        "concurrent-map-test.cc",
-    ],
-    deps=[
-        ":utils",
-    ],)
-cxx_test(
-    name="user-util-test",
-    srcs=[
-        "user-util-test.cc",
-    ],
-    deps=[
-        ":utils",
-    ],)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/bytes-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/bytes-util-test.cc b/hbase-native-client/utils/bytes-util-test.cc
deleted file mode 100644
index 4a49593..0000000
--- a/hbase-native-client/utils/bytes-util-test.cc
+++ /dev/null
@@ -1,69 +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 <folly/Logging.h>
-#include <gtest/gtest.h>
-#include <string>
-
-#include "utils/bytes-util.h"
-
-using hbase::BytesUtil;
-
-TEST(TestBytesUtil, TestToStringBinary) {
-  std::string empty{""};
-  EXPECT_EQ(empty, BytesUtil::ToStringBinary(empty));
-
-  std::string foo_bar{"foo bar"};
-  EXPECT_EQ(foo_bar, BytesUtil::ToStringBinary(foo_bar));
-
-  std::string foo_bar2{"foo bar_/!@#$%^&*(){}[]|1234567890"};
-  EXPECT_EQ(foo_bar2, BytesUtil::ToStringBinary(foo_bar2));
-
-  char zero = 0;
-  EXPECT_EQ("\\x00", BytesUtil::ToStringBinary(std::string{zero}));
-
-  char max = 255;
-  EXPECT_EQ("\\xFF", BytesUtil::ToStringBinary(std::string{max}));
-
-  EXPECT_EQ("\\x00\\xFF", BytesUtil::ToStringBinary(std::string{zero} + std::string{max}));
-
-  EXPECT_EQ("foo_\\x00\\xFF_bar",
-            BytesUtil::ToStringBinary("foo_" + std::string{zero} + std::string{max} + "_bar"));
-}
-
-TEST(TestBytesUtil, TestToStringToInt64) {
-  int64_t num = 761235;
-  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
-
-  num = -56125;
-  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
-
-  num = 0;
-  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
-}
-
-TEST(TestBytesUtil, TestCreateClosestRowAfter) {
-  std::string empty{""};
-  EXPECT_EQ(BytesUtil::CreateClosestRowAfter(empty), std::string{'\0'});
-
-  std::string foo{"foo"};
-  EXPECT_EQ(BytesUtil::CreateClosestRowAfter(foo), std::string{"foo"} + '\0');
-
-  EXPECT_EQ("f\\x00", BytesUtil::ToStringBinary(BytesUtil::CreateClosestRowAfter("f")));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/bytes-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/bytes-util.cc b/hbase-native-client/utils/bytes-util.cc
deleted file mode 100644
index 12037c3..0000000
--- a/hbase-native-client/utils/bytes-util.cc
+++ /dev/null
@@ -1,88 +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 "utils/bytes-util.h"
-
-#include <bits/stdc++.h>
-#include <boost/predef.h>
-#include <glog/logging.h>
-
-#include <memory>
-#include <string>
-
-namespace hbase {
-
-constexpr char BytesUtil::kHexChars[];
-
-std::string BytesUtil::ToString(int64_t val) {
-  std::string res;
-#if BOOST_ENDIAN_BIG_BYTE || BOOST_ENDIAN_BIG_WORD
-  for (int i = 7; i > 0; i--) {
-    res += (int8_t)(val & 0xffu);
-    val = val >> 8;
-  }
-  res += (int8_t)val;
-#else
-  int64_t mask = 0xff00000000000000u;
-  for (int i = 56; i >= 1; i -= 8) {
-    auto num = ((val & mask) >> i);
-    res += num;
-    mask = mask >> 8;
-  }
-  res += (val & 0xff);
-#endif
-  return res;
-}
-
-int64_t BytesUtil::ToInt64(std::string str) {
-  if (str.length() < 8) {
-    throw std::runtime_error("There are not enough bytes. Expected: 8, actual: " + str.length());
-  }
-  const unsigned char *bytes = reinterpret_cast<unsigned char *>(const_cast<char *>(str.c_str()));
-  int64_t l = 0;
-  for (int i = 0; i < 8; i++) {
-    l <<= 8;
-    l ^= bytes[i];
-  }
-  return l;
-}
-
-std::string BytesUtil::ToStringBinary(const std::string &b, size_t off, size_t len) {
-  std::string result;
-  // Just in case we are passed a 'len' that is > buffer length...
-  if (off >= b.size()) {
-    return result;
-  }
-  if (off + len > b.size()) {
-    len = b.size() - off;
-  }
-  for (size_t i = off; i < off + len; ++i) {
-    int32_t ch = b[i] & 0xFF;
-    if (ch >= ' ' && ch <= '~' && ch != '\\') {
-      result += ch;
-    } else {
-      result += "\\x";
-      result += kHexChars[ch / 0x10];
-      result += kHexChars[ch % 0x10];
-    }
-  }
-  return result;
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/bytes-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/bytes-util.h b/hbase-native-client/utils/bytes-util.h
deleted file mode 100644
index 6221bf0..0000000
--- a/hbase-native-client/utils/bytes-util.h
+++ /dev/null
@@ -1,68 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <string>
-
-namespace hbase {
-
-class BytesUtil {
- private:
-  static const constexpr char kHexChars[] = "0123456789ABCDEF";
-
- public:
-  static std::string ToStringBinary(const std::string& b) { return ToStringBinary(b, 0, b.size()); }
-  /**
-    * Write a printable representation of a byte array. Non-printable
-    * characters are hex escaped in the format \\x%02X, eg:
-    * \x00 \x05 etc
-    *
-    * @param b array to write out
-    * @param off offset to start at
-    * @param len length to write
-    * @return string output
-    */
-  static std::string ToStringBinary(const std::string& b, size_t off, size_t len);
-
-  static std::string ToString(int64_t value);
-
-  static int64_t ToInt64(std::string str);
-
-  static bool IsEmptyStartRow(const std::string& row) { return row == ""; }
-
-  static bool IsEmptyStopRow(const std::string& row) { return row == ""; }
-
-  static int32_t CompareTo(const std::string& a, const std::string& b) {
-    if (a < b) {
-      return -1;
-    }
-    if (a == b) {
-      return 0;
-    }
-    return 1;
-  }
-
-  /**
-   * Create the closest row after the specified row
-   */
-  static std::string CreateClosestRowAfter(std::string row) { return row.append(1, '\0'); }
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/concurrent-map-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/concurrent-map-test.cc b/hbase-native-client/utils/concurrent-map-test.cc
deleted file mode 100644
index 588bd08..0000000
--- a/hbase-native-client/utils/concurrent-map-test.cc
+++ /dev/null
@@ -1,36 +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 <folly/Logging.h>
-#include <gtest/gtest.h>
-#include <string>
-
-#include "utils/concurrent-map.h"
-
-using hbase::concurrent_map;
-
-TEST(TestConcurrentMap, TestFindAndErase) {
-  concurrent_map<std::string, std::string> map{500};
-
-  map.insert(std::make_pair("foo", "bar"));
-  auto prev = map.find_and_erase("foo");
-  ASSERT_EQ("bar", prev);
-
-  ASSERT_EQ(map.end(), map.find("foo"));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/concurrent-map.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/concurrent-map.h b/hbase-native-client/utils/concurrent-map.h
deleted file mode 100644
index aebca0d..0000000
--- a/hbase-native-client/utils/concurrent-map.h
+++ /dev/null
@@ -1,130 +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.
- *
- */
-
-#pragma once
-
-#include <map>
-#include <memory>
-#include <shared_mutex>
-#include <unordered_map>
-#include <utility>
-
-namespace hbase {
-
-/**
- * A concurrent version of std::unordered_map where we acquire a shared or exclusive
- * lock for operations. This is NOT a highly-concurrent and scalable implementation
- * since there is only one lock object.
- * Replace this with tbb::concurrent_unordered_map or similar.
- *
- * Concurrency here is different than in Java. For example, the iterators returned from
- * find() will not copy the key, value pairs.
- */
-template <typename K, typename V>
-class concurrent_map {
- public:
-  typedef K key_type;
-  typedef V mapped_type;
-  typedef std::pair<const key_type, mapped_type> value_type;
-  typedef typename std::unordered_map<K, V>::iterator iterator;
-  typedef typename std::unordered_map<K, V>::const_iterator const_iterator;
-
-  concurrent_map() : map_(), mutex_() {}
-  explicit concurrent_map(int32_t n) : map_(n), mutex_() {}
-
-  void insert(const value_type& value) {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    map_.insert(value);
-  }
-
-  /**
-   * Return the mapped object for this key. Be careful to not use the return reference
-   * to do assignment. I think it won't be thread safe
-   */
-  mapped_type& at(const key_type& key) {
-    std::shared_lock<std::shared_timed_mutex> lock(mutex_);
-    iterator where = map_.find(key);
-    if (where == end()) {
-      std::runtime_error("Key not found");
-    }
-    return where->second;
-  }
-
-  mapped_type& operator[](const key_type& key) {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    iterator where = map_.find(key);
-    if (where == end()) {
-      return map_[key];
-    }
-    return where->second;
-  }
-
-  /**
-   * Atomically finds the entry and removes it from the map, returning
-   * the previously associated value.
-   */
-  mapped_type find_and_erase(const K& key) {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    auto search = map_.find(key);
-    // It's an error if it's not there.
-    CHECK(search != end());
-    auto val = std::move(search->second);
-    map_.erase(key);
-    return val;
-  }
-
-  void erase(const K& key) {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    map_.erase(key);
-  }
-
-  iterator begin() { return map_.begin(); }
-
-  const_iterator begin() const { return map_.begin(); }
-
-  const_iterator cbegin() const { return map_.begin(); }
-
-  iterator end() { return map_.end(); }
-
-  const_iterator end() const { return map_.end(); }
-
-  const_iterator cend() const { return map_.end(); }
-
-  iterator find(const K& key) {
-    std::shared_lock<std::shared_timed_mutex> lock(mutex_);
-    return map_.find(key);
-  }
-
-  // TODO: find(), at() returning const_iterator
-
-  bool empty() const {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    return map_.empty();
-  }
-
-  void clear() {
-    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
-    map_.clear();
-  }
-
- private:
-  std::shared_timed_mutex mutex_;
-  std::unordered_map<K, V> map_;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/connection-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/connection-util.cc b/hbase-native-client/utils/connection-util.cc
deleted file mode 100644
index 76689bf..0000000
--- a/hbase-native-client/utils/connection-util.cc
+++ /dev/null
@@ -1,26 +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 "utils/connection-util.h"
-
-namespace hbase {
-
-const std::vector<uint32_t> ConnectionUtils::kRetryBackoff = {1,   2,   3,   5,   10,  20, 40,
-                                                              100, 100, 100, 100, 200, 200};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/connection-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/connection-util.h b/hbase-native-client/utils/connection-util.h
deleted file mode 100644
index f52c2f9..0000000
--- a/hbase-native-client/utils/connection-util.h
+++ /dev/null
@@ -1,62 +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.
- *
- */
-
-#pragma once
-
-#include <algorithm>
-#include <climits>
-#include <cstdlib>
-#include <memory>
-#include <vector>
-#include "utils/time-util.h"
-
-namespace hbase {
-class ConnectionUtils {
- public:
-  static int Retries2Attempts(const int& retries) {
-    return std::max(1, retries == INT_MAX ? INT_MAX : retries + 1);
-  }
-
-  /* Add a delta to avoid timeout immediately after a retry sleeping. */
-  static const uint64_t kSleepDeltaNs = 1000000;
-
-  static const std::vector<uint32_t> kRetryBackoff;
-  /**
-   * Calculate pause time. Built on {@link kRetryBackoff}.
-   * @param pause time to pause
-   * @param tries amount of tries
-   * @return How long to wait after <code>tries</code> retries
-   */
-  static int64_t GetPauseTime(const int64_t& pause, const int32_t& tries) {
-    int32_t ntries = tries;
-    if (static_cast<size_t>(ntries) >= kRetryBackoff.size()) {
-      ntries = kRetryBackoff.size() - 1;
-    }
-    if (ntries < 0) {
-      ntries = 0;
-    }
-
-    int64_t normal_pause = pause * kRetryBackoff[ntries];
-    // 1% possible jitter
-    float r = static_cast<float>(std::rand()) / static_cast<float>(RAND_MAX);
-    int64_t jitter = (int64_t)(normal_pause * r * 0.01f);
-    return normal_pause + jitter;
-  }
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/optional.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/optional.h b/hbase-native-client/utils/optional.h
deleted file mode 100644
index a05eab5..0000000
--- a/hbase-native-client/utils/optional.h
+++ /dev/null
@@ -1,32 +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.
- *
- */
-
-#pragma once
-
-#include <experimental/optional>
-
-namespace hbase {
-
-/**
- * An optional value that may or may not be present.
- */
-template <class T>
-using optional = std::experimental::optional<T>;
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/sys-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/sys-util.h b/hbase-native-client/utils/sys-util.h
deleted file mode 100644
index 68f00d7..0000000
--- a/hbase-native-client/utils/sys-util.h
+++ /dev/null
@@ -1,39 +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.
- *
- */
-
-#pragma once
-
-#include <type_traits>
-
-namespace hbase {
-
-class SysUtil {
- public:
-  template <class BASE, typename DERIVED>
-  static constexpr bool InstanceOf(const DERIVED& object) {
-    return !dynamic_cast<const BASE*>(&object);
-  }
-
-  template <typename BASE, typename DERIVED>
-  static constexpr bool InstanceOf() {
-    return std::is_base_of<BASE, DERIVED>();
-  }
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/time-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/time-util.h b/hbase-native-client/utils/time-util.h
deleted file mode 100644
index 165e9f1..0000000
--- a/hbase-native-client/utils/time-util.h
+++ /dev/null
@@ -1,71 +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.
- *
- */
-
-#pragma once
-
-#include <chrono>
-#include <string>
-
-namespace hbase {
-
-class TimeUtil {
- public:
-  static inline int64_t ToMillis(const int64_t& nanos) {
-    return std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::nanoseconds(nanos))
-        .count();
-  }
-
-  static inline std::chrono::milliseconds ToMillis(const std::chrono::nanoseconds& nanos) {
-    return std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::nanoseconds(nanos));
-  }
-
-  static inline std::chrono::nanoseconds ToNanos(const std::chrono::milliseconds& millis) {
-    return std::chrono::duration_cast<std::chrono::nanoseconds>(millis);
-  }
-
-  static inline std::chrono::nanoseconds MillisToNanos(const int64_t& millis) {
-    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::milliseconds(millis));
-  }
-
-  static inline std::chrono::nanoseconds SecondsToNanos(const int64_t& secs) {
-    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::seconds(secs));
-  }
-
-  static inline std::string ToMillisStr(const std::chrono::nanoseconds& nanos) {
-    return std::to_string(std::chrono::duration_cast<std::chrono::milliseconds>(nanos).count());
-  }
-
-  static inline int64_t GetNowNanos() {
-    auto duration = std::chrono::high_resolution_clock::now().time_since_epoch();
-    return std::chrono::duration_cast<std::chrono::nanoseconds>(duration).count();
-  }
-
-  static inline int64_t ElapsedMillis(const int64_t& start_ns) {
-    return std::chrono::duration_cast<std::chrono::milliseconds>(
-               std::chrono::nanoseconds(GetNowNanos() - start_ns))
-        .count();
-  }
-
-  static inline std::string ElapsedMillisStr(const int64_t& start_ns) {
-    return std::to_string(std::chrono::duration_cast<std::chrono::milliseconds>(
-                              std::chrono::nanoseconds(GetNowNanos() - start_ns))
-                              .count());
-  }
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/user-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/user-util-test.cc b/hbase-native-client/utils/user-util-test.cc
deleted file mode 100644
index aa3fa45..0000000
--- a/hbase-native-client/utils/user-util-test.cc
+++ /dev/null
@@ -1,35 +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 <folly/Logging.h>
-#include <gtest/gtest.h>
-#include <string>
-
-#include "utils/user-util.h"
-
-using namespace std;
-using namespace hbase;
-
-TEST(TestUserUtil, TestGetSomething) {
-  UserUtil u_util;
-  string name = u_util.user_name(false);
-
-  // TODO shell out to whoami to check this.
-  ASSERT_GT(name.length(), 0);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/user-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/user-util.cc b/hbase-native-client/utils/user-util.cc
deleted file mode 100644
index 092d54c..0000000
--- a/hbase-native-client/utils/user-util.cc
+++ /dev/null
@@ -1,77 +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 "utils/user-util.h"
-
-#include <folly/Logging.h>
-#include <krb5/krb5.h>
-#include <pwd.h>
-#include <sys/types.h>
-#include <unistd.h>
-
-namespace hbase {
-
-UserUtil::UserUtil() : once_flag_{} {}
-
-std::string UserUtil::user_name(bool secure) {
-  std::call_once(once_flag_, [this, secure]() { compute_user_name(secure); });
-  return user_name_;
-}
-
-void UserUtil::compute_user_name(bool secure) {
-  // According to the man page of getpwuid
-  // this should never be free'd
-  //
-  // So yeah a raw pointer with no ownership....
-  struct passwd *passwd = getpwuid(getuid());
-
-  // make sure that we got something.
-  if (passwd && passwd->pw_name) {
-    user_name_ = std::string{passwd->pw_name};
-  }
-  if (!secure) return;
-  krb5_context ctx;
-  krb5_error_code ret = krb5_init_context(&ctx);
-  if (ret != 0) {
-    throw std::runtime_error("cannot init krb ctx " + std::to_string(ret));
-  }
-  krb5_ccache ccache;
-  ret = krb5_cc_default(ctx, &ccache);
-  if (ret != 0) {
-    throw std::runtime_error("cannot get default cache " + std::to_string(ret));
-  }
-  // Here is sample principal: hbase/23a03935850c@EXAMPLE.COM
-  // There may be one (user) or two (user/host) components before the @ sign
-  krb5_principal princ;
-  ret = krb5_cc_get_principal(ctx, ccache, &princ);
-  if (ret != 0) {
-    throw std::runtime_error("cannot get default principal " + std::to_string(ret));
-  }
-  user_name_ = princ->data->data;
-  if (krb5_princ_size(ctx, princ) >= 2) {
-    user_name_ += "/";
-    user_name_ += static_cast<char *>(princ->data[1].data);
-  }
-  user_name_ += "@";
-  user_name_ += princ->realm.data;
-  VLOG(1) << "user " << user_name_;
-  krb5_free_principal(ctx, princ);
-  krb5_free_context(ctx);
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/utils/user-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/user-util.h b/hbase-native-client/utils/user-util.h
deleted file mode 100644
index 6258c85..0000000
--- a/hbase-native-client/utils/user-util.h
+++ /dev/null
@@ -1,54 +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.
- *
- */
-
-#pragma once
-
-#include <mutex>
-#include <string>
-
-namespace hbase {
-
-/**
- * @brief Class to help with user/group information.
- *
- * This class will get the current user, and information about them. It caches
- * the user information after the first invocation.
- */
-class UserUtil {
- public:
-  /**
-   * Constructor.
-   */
-  UserUtil();
-
-  /**
-   * Get the username of the user owning this process. This is thread safe and
-   * lockless for every invocation other than the first one.
-   */
-  std::string user_name(bool secure = false);
-
- private:
-  /**
-   * Compute the username. This will block.
-   */
-  void compute_user_name(bool secure);
-  std::once_flag once_flag_;
-  std::string user_name_;
-};
-}  // namespace hbase


[09/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-test.cc b/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-test.cc
new file mode 100644
index 0000000..b0f4afb
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-test.cc
@@ -0,0 +1,577 @@
+/*
+ * 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 <folly/Logging.h>
+#include <folly/Memory.h>
+#include <folly/futures/Future.h>
+#include <folly/io/async/EventBase.h>
+#include <folly/io/async/ScopedEventBaseThread.h>
+#include <gtest/gtest.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+
+#include <chrono>
+#include <functional>
+#include <string>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-batch-rpc-retrying-caller.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/async-rpc-retrying-caller-factory.h"
+#include "hbase/client/client.h"
+#include "hbase/client/connection-configuration.h"
+#include "hbase/client/keyvalue-codec.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/result.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/test-util/test-util.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::AsyncRpcRetryingCallerFactory;
+using hbase::AsyncConnection;
+using hbase::AsyncRegionLocator;
+using hbase::ConnectionConfiguration;
+using hbase::Configuration;
+using hbase::HBaseRpcController;
+using hbase::RegionLocation;
+using hbase::RegionLocateType;
+using hbase::RpcClient;
+using hbase::RequestConverter;
+using hbase::ResponseConverter;
+using hbase::Put;
+using hbase::TimeUtil;
+using hbase::Client;
+using hbase::security::User;
+
+using std::chrono::nanoseconds;
+using std::chrono::milliseconds;
+
+using namespace hbase;
+
+using folly::exception_wrapper;
+
+class AsyncBatchRpcRetryTest : public ::testing::Test {
+ public:
+  static std::unique_ptr<hbase::TestUtil> test_util;
+  static std::string tableName;
+
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util = std::make_unique<hbase::TestUtil>();
+    test_util->StartMiniCluster(2);
+    std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
+                                  "test500", "test600", "test700", "test800", "test900"};
+    tableName = "split-table1";
+    test_util->CreateTable(tableName, "d", keys);
+  }
+};
+std::unique_ptr<hbase::TestUtil> AsyncBatchRpcRetryTest::test_util = nullptr;
+std::string AsyncBatchRpcRetryTest::tableName;
+
+class AsyncRegionLocatorBase : public AsyncRegionLocator {
+ public:
+  AsyncRegionLocatorBase() {}
+  explicit AsyncRegionLocatorBase(std::shared_ptr<RegionLocation> region_location)
+      : region_location_(region_location) {}
+  virtual ~AsyncRegionLocatorBase() = default;
+
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(const hbase::pb::TableName &,
+                                                                     const std::string &row,
+                                                                     const RegionLocateType,
+                                                                     const int64_t) override {
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    promise.setValue(region_locations_.at(row));
+    return promise.getFuture();
+  }
+
+  virtual void set_region_location(std::shared_ptr<RegionLocation> region_location) {
+    region_location_ = region_location;
+  }
+
+  virtual void set_region_location(
+      const std::map<std::string, std::shared_ptr<RegionLocation>> &reg_locs) {
+    for (auto reg_loc : reg_locs) {
+      region_locations_[reg_loc.first] = reg_loc.second;
+    }
+  }
+
+  void UpdateCachedLocation(const RegionLocation &rl, const folly::exception_wrapper &ew) override {
+  }
+
+ protected:
+  std::shared_ptr<RegionLocation> region_location_;
+  std::map<std::string, std::shared_ptr<RegionLocation>> region_locations_;
+  std::map<std::string, uint32_t> mtries_;
+  std::map<std::string, uint32_t> mnum_fails_;
+
+  void InitRetryMaps(uint32_t num_fails) {
+    if (mtries_.size() == 0 && mnum_fails_.size() == 0) {
+      for (auto reg_loc : region_locations_) {
+        mtries_[reg_loc.first] = 0;
+        mnum_fails_[reg_loc.first] = num_fails;
+      }
+    }
+  }
+};
+
+class MockAsyncRegionLocator : public AsyncRegionLocatorBase {
+ public:
+  MockAsyncRegionLocator() : AsyncRegionLocatorBase() {}
+  explicit MockAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockAsyncRegionLocator() {}
+};
+
+class MockWrongRegionAsyncRegionLocator : public AsyncRegionLocatorBase {
+ private:
+  uint32_t counter_ = 0;
+  uint32_t num_fails_ = 0;
+  uint32_t tries_ = 0;
+
+ public:
+  explicit MockWrongRegionAsyncRegionLocator(uint32_t num_fails)
+      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
+  explicit MockWrongRegionAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockWrongRegionAsyncRegionLocator() {}
+
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) override {
+    InitRetryMaps(num_fails_);
+    auto &tries = mtries_[row];
+    auto &num_fails = mnum_fails_[row];
+    if (++tries > num_fails) {
+      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
+    }
+
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    /* set random region name, simulating invalid region */
+    auto result = std::make_shared<RegionLocation>("whatever-region-name",
+                                                   region_locations_.at(row)->region_info(),
+                                                   region_locations_.at(row)->server_name());
+    promise.setValue(result);
+    return promise.getFuture();
+  }
+};
+
+class MockFailingAsyncRegionLocator : public AsyncRegionLocatorBase {
+ private:
+  uint32_t tries_ = 0;
+  uint32_t num_fails_ = 0;
+  uint32_t counter_ = 0;
+
+ public:
+  explicit MockFailingAsyncRegionLocator(uint32_t num_fails)
+      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
+  explicit MockFailingAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockFailingAsyncRegionLocator() {}
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) override {
+    InitRetryMaps(num_fails_);
+    auto &tries = mtries_[row];
+    auto &num_fails = mnum_fails_[row];
+    if (++tries > num_fails) {
+      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
+    }
+
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    promise.setException(std::runtime_error{"Failed to look up region location"});
+    return promise.getFuture();
+  }
+};
+
+class MockAsyncConnection : public AsyncConnection,
+                            public std::enable_shared_from_this<MockAsyncConnection> {
+ public:
+  MockAsyncConnection(std::shared_ptr<ConnectionConfiguration> conn_conf,
+                      std::shared_ptr<folly::HHWheelTimer> retry_timer,
+                      std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                      std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                      std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor,
+                      std::shared_ptr<RpcClient> rpc_client,
+                      std::shared_ptr<AsyncRegionLocator> region_locator)
+      : conn_conf_(conn_conf),
+        retry_timer_(retry_timer),
+        cpu_executor_(cpu_executor),
+        io_executor_(io_executor),
+        retry_executor_(retry_executor),
+        rpc_client_(rpc_client),
+        region_locator_(region_locator) {}
+  ~MockAsyncConnection() {}
+  void Init() {
+    caller_factory_ =
+        std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
+  }
+
+  std::shared_ptr<Configuration> conf() override { return nullptr; }
+  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return conn_conf_; }
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
+    return caller_factory_;
+  }
+  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
+  std::shared_ptr<AsyncRegionLocator> region_locator() override { return region_locator_; }
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
+    return retry_executor_;
+  }
+
+  void Close() override {
+    retry_timer_->destroy();
+    retry_executor_->stop();
+    io_executor_->stop();
+    cpu_executor_->stop();
+  }
+  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
+    return std::make_shared<HBaseRpcController>();
+  }
+
+ private:
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<ConnectionConfiguration> conn_conf_;
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
+  std::shared_ptr<RpcClient> rpc_client_;
+  std::shared_ptr<AsyncRegionLocator> region_locator_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
+};
+
+class MockRawAsyncTableImpl {
+ public:
+  explicit MockRawAsyncTableImpl(std::shared_ptr<MockAsyncConnection> conn,
+                                 std::shared_ptr<hbase::pb::TableName> tn)
+      : conn_(conn), tn_(tn) {}
+  virtual ~MockRawAsyncTableImpl() = default;
+
+  /* implement this in real RawAsyncTableImpl. */
+  template <typename REQ, typename RESP>
+  folly::Future<std::vector<folly::Try<RESP>>> Batch(const std::vector<REQ> &rows) {
+    /* init request caller builder */
+    auto builder = conn_->caller_factory()->Batch<REQ, RESP>();
+
+    /* call with retry to get result */
+    auto async_caller =
+        builder->table(tn_)
+            ->actions(std::make_shared<std::vector<REQ>>(rows))
+            ->rpc_timeout(conn_->connection_conf()->read_rpc_timeout())
+            ->operation_timeout(conn_->connection_conf()->operation_timeout())
+            ->pause(conn_->connection_conf()->pause())
+            ->max_attempts(conn_->connection_conf()->max_retries())
+            ->start_log_errors_count(conn_->connection_conf()->start_log_errors_count())
+            ->Build();
+
+    return async_caller->Call().then([async_caller](auto r) { return r; });
+  }
+
+ private:
+  std::shared_ptr<MockAsyncConnection> conn_;
+  std::shared_ptr<hbase::pb::TableName> tn_;
+};
+
+std::shared_ptr<MockAsyncConnection> getAsyncConnection(
+    Client &client, uint32_t operation_timeout_millis, uint32_t tries,
+    std::shared_ptr<AsyncRegionLocatorBase> region_locator) {
+  /* init region location and rpc channel */
+  auto cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
+  auto io_executor_ = client.async_connection()->io_executor();
+  auto retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
+  auto codec = std::make_shared<hbase::KeyValueCodec>();
+  auto rpc_client = std::make_shared<RpcClient>(io_executor_, cpu_executor_, codec,
+                                                AsyncBatchRpcRetryTest::test_util->conf());
+  std::shared_ptr<folly::HHWheelTimer> retry_timer =
+      folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
+
+  /* init connection configuration */
+  auto connection_conf = std::make_shared<ConnectionConfiguration>(
+      TimeUtil::SecondsToNanos(20),                       // connect_timeout
+      TimeUtil::MillisToNanos(operation_timeout_millis),  // operation_timeout
+      TimeUtil::SecondsToNanos(60),                       // rpc_timeout
+      TimeUtil::MillisToNanos(100),                       // pause
+      tries,                                              // max retries
+      1);                                                 // start log errors count
+
+  return std::make_shared<MockAsyncConnection>(connection_conf, retry_timer, cpu_executor_,
+                                               io_executor_, retry_executor_, rpc_client,
+                                               region_locator);
+}
+
+template <typename ACTION>
+std::vector<std::shared_ptr<hbase::Row>> getRows(std::vector<ACTION> actions) {
+  std::vector<std::shared_ptr<hbase::Row>> rows;
+  for (auto action : actions) {
+    std::shared_ptr<hbase::Row> srow = std::make_shared<ACTION>(action);
+    rows.push_back(srow);
+  }
+  return rows;
+}
+
+template <typename REQ, typename RESP>
+std::vector<std::shared_ptr<hbase::Result>> getResults(std::vector<REQ> &actions,
+                                                       std::vector<folly::Try<RESP>> &tresults) {
+  std::vector<std::shared_ptr<hbase::Result>> results{};
+  uint64_t num = 0;
+  for (auto tresult : tresults) {
+    if (tresult.hasValue()) {
+      results.push_back(tresult.value());
+    } else if (tresult.hasException()) {
+      folly::exception_wrapper ew = tresult.exception();
+      LOG(ERROR) << "Caught exception:- " << ew.what().toStdString() << " for "
+                 << actions[num].row();
+      throw ew;
+    }
+    ++num;
+  }
+  return results;
+}
+
+template <typename ACTION>
+std::map<std::string, std::shared_ptr<RegionLocation>> getRegionLocationsAndActions(
+    uint64_t num_rows, std::vector<ACTION> &actions, std::shared_ptr<Table> table) {
+  std::map<std::string, std::shared_ptr<RegionLocation>> region_locations;
+  for (uint64_t i = 0; i < num_rows; ++i) {
+    auto row = "test" + std::to_string(i);
+    ACTION action(row);
+    actions.push_back(action);
+    region_locations[row] = table->GetRegionLocation(row);
+  }
+  return region_locations;
+}
+
+void runMultiGets(std::shared_ptr<AsyncRegionLocatorBase> region_locator,
+                  const std::string &table_name, bool split_regions, uint32_t tries = 3,
+                  uint32_t operation_timeout_millis = 600000, uint64_t num_rows = 1000) {
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(AsyncBatchRpcRetryTest::tableName);
+
+  // Create a client
+  Client client(*AsyncBatchRpcRetryTest::test_util->conf());
+
+  // Get connection to HBase Table
+  std::shared_ptr<Table> table = client.Table(tn);
+
+  for (uint64_t i = 0; i < num_rows; i++) {
+    table->Put(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
+                                                         "value" + std::to_string(i)));
+  }
+  std::vector<hbase::Get> gets;
+  auto region_locations = getRegionLocationsAndActions<hbase::Get>(num_rows, gets, table);
+
+  /* set region locator */
+  region_locator->set_region_location(region_locations);
+
+  /* init hbase client connection */
+  auto conn = getAsyncConnection(client, operation_timeout_millis, tries, region_locator);
+  conn->Init();
+
+  /* init retry caller factory */
+  auto tableImpl =
+      std::make_shared<MockRawAsyncTableImpl>(conn, std::make_shared<hbase::pb::TableName>(tn));
+
+  std::vector<std::shared_ptr<hbase::Row>> rows = getRows<hbase::Get>(gets);
+  auto tresults = tableImpl->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(rows).get(
+      milliseconds(operation_timeout_millis));
+  ASSERT_TRUE(!tresults.empty()) << "tresults shouldn't be empty.";
+
+  auto results = getResults<hbase::Get, std::shared_ptr<Result>>(gets, tresults);
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!results.empty()) << "Results shouldn't be empty.";
+  uint32_t i = 0;
+  for (; i < num_rows; ++i) {
+    ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row()
+                                        << " must not be empty";
+    EXPECT_EQ("test" + std::to_string(i), results[i]->Row());
+    EXPECT_EQ("value" + std::to_string(i), results[i]->Value("d", std::to_string(i)).value());
+  }
+
+  table->Close();
+  client.Close();
+  conn->Close();
+}
+
+void runMultiPuts(std::shared_ptr<AsyncRegionLocatorBase> region_locator,
+                  const std::string &table_name, bool split_regions, uint32_t tries = 3,
+                  uint32_t operation_timeout_millis = 600000, uint32_t num_rows = 1000) {
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(AsyncBatchRpcRetryTest::tableName);
+
+  // Create a client
+  Client client(*AsyncBatchRpcRetryTest::test_util->conf());
+
+  // Get connection to HBase Table
+  std::shared_ptr<Table> table = client.Table(tn);
+
+  std::vector<hbase::Put> puts;
+  auto region_locations = getRegionLocationsAndActions<hbase::Put>(num_rows, puts, table);
+
+  /* set region locator */
+  region_locator->set_region_location(region_locations);
+
+  /* init hbase client connection */
+  auto conn = getAsyncConnection(client, operation_timeout_millis, tries, region_locator);
+  conn->Init();
+
+  /* init retry caller factory */
+  auto tableImpl =
+      std::make_shared<MockRawAsyncTableImpl>(conn, std::make_shared<hbase::pb::TableName>(tn));
+
+  std::vector<std::shared_ptr<hbase::Row>> rows = getRows<hbase::Put>(puts);
+  auto tresults = tableImpl->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(rows).get(
+      milliseconds(operation_timeout_millis));
+  ASSERT_TRUE(!tresults.empty()) << "tresults shouldn't be empty.";
+
+  auto results = getResults<hbase::Put, std::shared_ptr<Result>>(puts, tresults);
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!results.empty()) << "Results shouldn't be empty.";
+
+  table->Close();
+  client.Close();
+  conn->Close();
+}
+
+// Test successful case
+TEST_F(AsyncBatchRpcRetryTest, MultiGets) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockAsyncRegionLocator>());
+  runMultiGets(region_locator, "table1", false);
+}
+
+// Tests the RPC failing 3 times, then succeeding
+TEST_F(AsyncBatchRpcRetryTest, HandleException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
+  runMultiGets(region_locator, "table2", false, 5);
+}
+
+// Tests the RPC failing 4 times, throwing an exception
+TEST_F(AsyncBatchRpcRetryTest, FailWithException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
+  EXPECT_ANY_THROW(runMultiGets(region_locator, "table3", false));
+}
+
+// Tests the region location lookup failing 3 times, then succeeding
+TEST_F(AsyncBatchRpcRetryTest, HandleExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(3));
+  runMultiGets(region_locator, "table4", false);
+}
+
+// Tests the region location lookup failing 5 times, throwing an exception
+TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(4));
+  EXPECT_ANY_THROW(runMultiGets(region_locator, "table5", false, 3));
+}
+
+// Tests hitting operation timeout, thus not retrying anymore
+TEST_F(AsyncBatchRpcRetryTest, FailWithOperationTimeout) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(6));
+  EXPECT_ANY_THROW(runMultiGets(region_locator, "table6", false, 5, 100, 1000));
+}
+
+//////////////////////
+// Test successful case
+TEST_F(AsyncBatchRpcRetryTest, MultiPuts) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockAsyncRegionLocator>());
+  runMultiPuts(region_locator, "table1", false);
+}
+
+// Tests the RPC failing 3 times, then succeeding
+TEST_F(AsyncBatchRpcRetryTest, PutsHandleException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
+  runMultiPuts(region_locator, "table2", false, 5);
+}
+
+// Tests the RPC failing 4 times, throwing an exception
+TEST_F(AsyncBatchRpcRetryTest, PutsFailWithException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
+  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table3", false));
+}
+
+// Tests the region location lookup failing 3 times, then succeeding
+TEST_F(AsyncBatchRpcRetryTest, PutsHandleExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(3));
+  runMultiPuts(region_locator, "table4", false);
+}
+
+// Tests the region location lookup failing 5 times, throwing an exception
+TEST_F(AsyncBatchRpcRetryTest, PutsFailWithExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(4));
+  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table5", false, 3));
+}
+
+// Tests hitting operation timeout, thus not retrying anymore
+TEST_F(AsyncBatchRpcRetryTest, PutsFailWithOperationTimeout) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(6));
+  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table6", false, 5, 100, 1000));
+}
+
+ // Test successful case
+ TEST_F(AsyncBatchRpcRetryTest, MultiGetsSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockAsyncRegionLocator>());
+ runMultiGets(region_locator, "table7", true);
+ }
+
+ // Tests the RPC failing 3 times, then succeeding
+ TEST_F(AsyncBatchRpcRetryTest, HandleExceptionSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
+ runMultiGets(region_locator, "table8", true, 5);
+ }
+
+ // Tests the RPC failing 4 times, throwing an exception
+ TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
+ EXPECT_ANY_THROW(runMultiGets(region_locator, "table9", true));
+ }
+
+ // Tests the region location lookup failing 3 times, then succeeding
+ TEST_F(AsyncBatchRpcRetryTest, HandleExceptionFromRegionLocationLookupSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockFailingAsyncRegionLocator>(3));
+ runMultiGets(region_locator, "table10", true);
+ }
+
+ // Tests the region location lookup failing 5 times, throwing an exception
+ TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionFromRegionLocationLookupSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockFailingAsyncRegionLocator>(4));
+ EXPECT_ANY_THROW(runMultiGets(region_locator, "table11", true, 3));
+ }
+
+ // Tests hitting operation timeout, thus not retrying anymore
+ TEST_F(AsyncBatchRpcRetryTest, FailWithOperationTimeoutSplitRegions) {
+ std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+ std::make_shared<MockFailingAsyncRegionLocator>(6));
+ EXPECT_ANY_THROW(runMultiGets(region_locator, "table12", true, 5, 100, 1000));
+ }

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-client-scanner.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-client-scanner.cc b/hbase-native-client/src/hbase/client/async-client-scanner.cc
new file mode 100644
index 0000000..50c01ee
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-client-scanner.cc
@@ -0,0 +1,142 @@
+/*
+ * 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 "hbase/client/async-client-scanner.h"
+
+#include <algorithm>
+#include <iterator>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+
+AsyncClientScanner::AsyncClientScanner(
+    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<Scan> scan,
+    std::shared_ptr<pb::TableName> table_name, std::shared_ptr<RawScanResultConsumer> consumer,
+    nanoseconds pause, uint32_t max_retries, nanoseconds scan_timeout_nanos,
+    nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
+    : conn_(conn),
+      scan_(scan),
+      table_name_(table_name),
+      consumer_(consumer),
+      pause_(pause),
+      max_retries_(max_retries),
+      scan_timeout_nanos_(scan_timeout_nanos),
+      rpc_timeout_nanos_(rpc_timeout_nanos),
+      start_log_errors_count_(start_log_errors_count) {
+  results_cache_ = std::make_shared<ScanResultCache>();
+  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
+}
+
+void AsyncClientScanner::Start() { OpenScanner(); }
+
+folly::Future<std::shared_ptr<OpenScannerResponse>> AsyncClientScanner::CallOpenScanner(
+    std::shared_ptr<hbase::RpcClient> rpc_client,
+    std::shared_ptr<hbase::HBaseRpcController> controller,
+    std::shared_ptr<hbase::RegionLocation> loc) {
+  open_scanner_tries_++;
+
+  auto preq = RequestConverter::ToScanRequest(*scan_, loc->region_name(), scan_->Caching(), false);
+
+  auto self(shared_from_this());
+  VLOG(5) << "Calling RPC Client to open the scanner for region:" << loc->DebugString();
+  return rpc_client
+      ->AsyncCall(loc->server_name().host_name(), loc->server_name().port(), std::move(preq),
+                  security::User::defaultUser(), "ClientService")
+      .then([self, loc, controller, rpc_client](const std::unique_ptr<Response>& presp) {
+        VLOG(5) << "Scan Response:" << presp->DebugString();
+        return std::make_shared<OpenScannerResponse>(rpc_client, presp, loc, controller);
+      });
+}
+
+void AsyncClientScanner::OpenScanner() {
+  auto self(shared_from_this());
+  open_scanner_tries_ = 1;
+
+  auto caller = conn_->caller_factory()
+                    ->Single<std::shared_ptr<OpenScannerResponse>>()
+                    ->table(table_name_)
+                    ->row(scan_->StartRow())
+                    ->locate_type(GetLocateType(*scan_))
+                    ->rpc_timeout(rpc_timeout_nanos_)
+                    ->operation_timeout(scan_timeout_nanos_)
+                    ->pause(pause_)
+                    ->max_retries(max_retries_)
+                    ->start_log_errors_count(start_log_errors_count_)
+                    ->action([&](std::shared_ptr<hbase::HBaseRpcController> controller,
+                                 std::shared_ptr<hbase::RegionLocation> loc,
+                                 std::shared_ptr<hbase::RpcClient> rpc_client)
+                                 -> folly::Future<std::shared_ptr<OpenScannerResponse>> {
+                                   return CallOpenScanner(rpc_client, controller, loc);
+                                 })
+                    ->Build();
+
+  caller->Call()
+      .then([this, self](std::shared_ptr<OpenScannerResponse> resp) {
+        VLOG(3) << "Opened scanner with id:" << resp->scan_resp_->scanner_id()
+                << ", region:" << resp->region_location_->DebugString() << ", starting scan";
+        StartScan(resp);
+      })
+      .onError([this, self](const folly::exception_wrapper& e) {
+        VLOG(3) << "Open scan request received error:" << e.what();
+        consumer_->OnError(e);
+      })
+      .then([caller, self](const auto r) { return r; });
+}
+
+void AsyncClientScanner::StartScan(std::shared_ptr<OpenScannerResponse> resp) {
+  auto self(shared_from_this());
+  auto caller = conn_->caller_factory()
+                    ->Scan()
+                    ->scanner_id(resp->scan_resp_->scanner_id())
+                    ->region_location(resp->region_location_)
+                    ->scanner_lease_timeout(TimeUtil::MillisToNanos(resp->scan_resp_->ttl()))
+                    ->scan(scan_)
+                    ->rpc_client(resp->rpc_client_)
+                    ->consumer(consumer_)
+                    ->results_cache(results_cache_)
+                    ->rpc_timeout(rpc_timeout_nanos_)
+                    ->scan_timeout(scan_timeout_nanos_)
+                    ->pause(pause_)
+                    ->max_retries(max_retries_)
+                    ->start_log_errors_count(start_log_errors_count_)
+                    ->Build();
+
+  caller->Start(resp->controller_, resp->scan_resp_, resp->cell_scanner_)
+      .then([caller, self](const bool has_more) {
+        if (has_more) {
+          // open the next scanner on the next region.
+          self->OpenScanner();
+        } else {
+          self->consumer_->OnComplete();
+        }
+      })
+      .onError([caller, self](const folly::exception_wrapper& e) { self->consumer_->OnError(e); })
+      .then([caller, self](const auto r) { return r; });
+}
+
+RegionLocateType AsyncClientScanner::GetLocateType(const Scan& scan) {
+  // TODO: In C++, there is no Scan::IncludeStartRow() and Scan::IncludeStopRow().
+  // When added, this method should be modified to return other RegionLocateTypes
+  // (see ConnectionUtils.java #getLocateType())
+  // TODO: When reversed scans are implemented, return other RegionLocateTypes
+  return RegionLocateType::kCurrent;
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-connection.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-connection.cc b/hbase-native-client/src/hbase/client/async-connection.cc
new file mode 100644
index 0000000..f1bdebc
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-connection.cc
@@ -0,0 +1,70 @@
+/*
+ * 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 "hbase/client/async-connection.h"
+#include "hbase/client/async-rpc-retrying-caller-factory.h"
+
+namespace hbase {
+
+void AsyncConnectionImpl::Init() {
+  connection_conf_ = std::make_shared<hbase::ConnectionConfiguration>(*conf_);
+  // start thread pools
+  auto io_threads = conf_->GetInt(kClientIoThreadPoolSize, sysconf(_SC_NPROCESSORS_ONLN));
+  auto cpu_threads = conf_->GetInt(kClientCpuThreadPoolSize, 2 * sysconf(_SC_NPROCESSORS_ONLN));
+  cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(cpu_threads);
+  io_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(io_threads);
+  /*
+   * We need a retry_executor for a thread pool of size 1 due to a possible bug in wangle/folly.
+   * Otherwise, Assertion 'isInEventBaseThread()' always fails. See the comments
+   * in async-rpc-retrying-caller.cc.
+   */
+  retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
+  retry_timer_ = folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
+
+  std::shared_ptr<Codec> codec = nullptr;
+  if (conf_->Get(kRpcCodec, hbase::KeyValueCodec::kJavaClassName) ==
+      std::string(KeyValueCodec::kJavaClassName)) {
+    codec = std::make_shared<hbase::KeyValueCodec>();
+  } else {
+    LOG(WARNING) << "Not using RPC Cell Codec";
+  }
+  rpc_client_ = std::make_shared<hbase::RpcClient>(io_executor_, cpu_executor_, codec, conf_,
+                                                   connection_conf_->connect_timeout());
+  location_cache_ = std::make_shared<hbase::LocationCache>(conf_, io_executor_, cpu_executor_,
+                                                           rpc_client_->connection_pool());
+  caller_factory_ =
+      std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
+}
+
+// We can't have the threads continue running after everything is done
+// that leads to an error.
+AsyncConnectionImpl::~AsyncConnectionImpl() { Close(); }
+
+void AsyncConnectionImpl::Close() {
+  if (is_closed_) return;
+
+  cpu_executor_->stop();
+  io_executor_->stop();
+  retry_executor_->stop();
+  retry_timer_->destroy();
+  if (rpc_client_.get()) rpc_client_->Close();
+  is_closed_ = true;
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-rpc-retrying-caller-factory.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-rpc-retrying-caller-factory.cc b/hbase-native-client/src/hbase/client/async-rpc-retrying-caller-factory.cc
new file mode 100644
index 0000000..42b4eac
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-rpc-retrying-caller-factory.cc
@@ -0,0 +1,22 @@
+/*
+ * 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 "hbase/client/async-rpc-retrying-caller-factory.h"
+
+namespace hbase {}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-rpc-retrying-caller.cc b/hbase-native-client/src/hbase/client/async-rpc-retrying-caller.cc
new file mode 100644
index 0000000..4c39f05
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-rpc-retrying-caller.cc
@@ -0,0 +1,230 @@
+/*
+ * 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 "hbase/client/async-rpc-retrying-caller.h"
+
+#include <folly/Conv.h>
+#include <folly/ExceptionWrapper.h>
+#include <folly/Format.h>
+#include <folly/Logging.h>
+#include <folly/Unit.h>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/result.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/utils/connection-util.h"
+#include "hbase/utils/sys-util.h"
+#include "hbase/utils/time-util.h"
+
+using folly::exception_wrapper;
+
+namespace hbase {
+
+template <typename RESP>
+AsyncSingleRequestRpcRetryingCaller<RESP>::AsyncSingleRequestRpcRetryingCaller(
+    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
+    std::shared_ptr<hbase::pb::TableName> table_name, const std::string& row,
+    RegionLocateType locate_type, Callable<RESP> callable, std::chrono::nanoseconds pause,
+    uint32_t max_retries, std::chrono::nanoseconds operation_timeout_nanos,
+    std::chrono::nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
+    : conn_(conn),
+      retry_timer_(retry_timer),
+      table_name_(table_name),
+      row_(row),
+      locate_type_(locate_type),
+      callable_(callable),
+      pause_(pause),
+      max_retries_(max_retries),
+      operation_timeout_nanos_(operation_timeout_nanos),
+      rpc_timeout_nanos_(rpc_timeout_nanos),
+      start_log_errors_count_(start_log_errors_count),
+      promise_(std::make_shared<folly::Promise<RESP>>()),
+      tries_(1) {
+  controller_ = conn_->CreateRpcController();
+  start_ns_ = TimeUtil::GetNowNanos();
+  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
+  exceptions_ = std::make_shared<std::vector<ThrowableWithExtraContext>>();
+}
+
+template <typename RESP>
+AsyncSingleRequestRpcRetryingCaller<RESP>::~AsyncSingleRequestRpcRetryingCaller() {}
+
+template <typename RESP>
+folly::Future<RESP> AsyncSingleRequestRpcRetryingCaller<RESP>::Call() {
+  auto f = promise_->getFuture();
+  LocateThenCall();
+  return f;
+}
+
+template <typename RESP>
+void AsyncSingleRequestRpcRetryingCaller<RESP>::LocateThenCall() {
+  int64_t locate_timeout_ns;
+  if (operation_timeout_nanos_.count() > 0) {
+    locate_timeout_ns = RemainingTimeNs();
+    if (locate_timeout_ns <= 0) {
+      CompleteExceptionally();
+      return;
+    }
+  } else {
+    locate_timeout_ns = -1L;
+  }
+
+  conn_->region_locator()
+      ->LocateRegion(*table_name_, row_, locate_type_, locate_timeout_ns)
+      .then([this](std::shared_ptr<RegionLocation> loc) { Call(*loc); })
+      .onError([this](const exception_wrapper& e) {
+        OnError(e,
+                [this, e]() -> std::string {
+                  return "Locate '" + row_ + "' in " + table_name_->namespace_() + "::" +
+                         table_name_->qualifier() + " failed with e.what()=" +
+                         e.what().toStdString() + ", tries = " + std::to_string(tries_) +
+                         ", maxAttempts = " + std::to_string(max_attempts_) + ", timeout = " +
+                         TimeUtil::ToMillisStr(operation_timeout_nanos_) + " ms, time elapsed = " +
+                         TimeUtil::ElapsedMillisStr(this->start_ns_) + " ms";
+                },
+                [](const exception_wrapper& error) {});
+      });
+}
+
+template <typename RESP>
+void AsyncSingleRequestRpcRetryingCaller<RESP>::OnError(
+    const exception_wrapper& error, Supplier<std::string> err_msg,
+    Consumer<exception_wrapper> update_cached_location) {
+  ThrowableWithExtraContext twec(error, TimeUtil::GetNowNanos());
+  exceptions_->push_back(twec);
+  if (!ExceptionUtil::ShouldRetry(error) || tries_ >= max_retries_) {
+    CompleteExceptionally();
+    return;
+  }
+
+  if (tries_ > start_log_errors_count_) {
+    LOG(WARNING) << err_msg();
+  } else {
+    VLOG(1) << err_msg();
+  }
+
+  int64_t delay_ns;
+  if (operation_timeout_nanos_.count() > 0) {
+    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
+    if (max_delay_ns <= 0) {
+      CompleteExceptionally();
+      return;
+    }
+    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1));
+  } else {
+    delay_ns = ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1);
+  }
+  update_cached_location(error);
+  tries_++;
+
+  /*
+   * The HHWheelTimer::scheduleTimeout() fails with an assertion from
+   * EventBase::isInEventBaseThread() if we execute the schedule in a random thread, or one of
+   * the IOThreadPool threads (with num threads > 1). I think there is a bug there in using retry
+   * timer from IOThreadPool threads. It only works when executed from a single-thread pool
+   * (retry_executor() is). However, the scheduled "work" which is the LocateThenCall() should
+   * still happen in a thread pool, that is why we are submitting the work to the CPUThreadPool.
+   * IOThreadPool cannot be used without fixing the blocking call that we do at TCP connection
+   * establishment time (see ConnectionFactory::Connect()), otherwise, the IOThreadPool thread
+   * just hangs because it deadlocks itself.
+   */
+  conn_->retry_executor()->add([=]() {
+    retry_timer_->scheduleTimeoutFn(
+        [=]() { conn_->cpu_executor()->add([&]() { LocateThenCall(); }); },
+        std::chrono::milliseconds(TimeUtil::ToMillis(delay_ns)));
+  });
+}
+
+template <typename RESP>
+void AsyncSingleRequestRpcRetryingCaller<RESP>::Call(const RegionLocation& loc) {
+  int64_t call_timeout_ns;
+  if (operation_timeout_nanos_.count() > 0) {
+    call_timeout_ns = this->RemainingTimeNs();
+    if (call_timeout_ns <= 0) {
+      this->CompleteExceptionally();
+      return;
+    }
+    call_timeout_ns = std::min(call_timeout_ns, rpc_timeout_nanos_.count());
+  } else {
+    call_timeout_ns = rpc_timeout_nanos_.count();
+  }
+
+  std::shared_ptr<RpcClient> rpc_client;
+
+  rpc_client = conn_->rpc_client();
+
+  ResetController(controller_, call_timeout_ns);
+
+  // TODO: RegionLocation should propagate through these method chains as a shared_ptr.
+  // Otherwise, it may get deleted underneat us. We are just copying for now.
+  auto loc_ptr = std::make_shared<RegionLocation>(loc);
+  callable_(controller_, loc_ptr, rpc_client)
+      .then([loc_ptr, this](const RESP& resp) { this->promise_->setValue(std::move(resp)); })
+      .onError([&, loc_ptr, this](const exception_wrapper& e) {
+        OnError(
+            e,
+            [&, this, e]() -> std::string {
+              return "Call to " + folly::sformat("{0}:{1}", loc_ptr->server_name().host_name(),
+                                                 loc_ptr->server_name().port()) +
+                     " for '" + row_ + "' in " + loc_ptr->DebugString() + " of " +
+                     table_name_->namespace_() + "::" + table_name_->qualifier() +
+                     " failed with e.what()=" + e.what().toStdString() + ", tries = " +
+                     std::to_string(tries_) + ", maxAttempts = " + std::to_string(max_attempts_) +
+                     ", timeout = " + TimeUtil::ToMillisStr(this->operation_timeout_nanos_) +
+                     " ms, time elapsed = " + TimeUtil::ElapsedMillisStr(this->start_ns_) + " ms";
+            },
+            [&, this](const exception_wrapper& error) {
+              conn_->region_locator()->UpdateCachedLocation(*loc_ptr, error);
+            });
+      });
+}
+
+template <typename RESP>
+void AsyncSingleRequestRpcRetryingCaller<RESP>::CompleteExceptionally() {
+  this->promise_->setException(RetriesExhaustedException(tries_ - 1, exceptions_));
+}
+
+template <typename RESP>
+int64_t AsyncSingleRequestRpcRetryingCaller<RESP>::RemainingTimeNs() {
+  return operation_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
+}
+
+template <typename RESP>
+void AsyncSingleRequestRpcRetryingCaller<RESP>::ResetController(
+    std::shared_ptr<HBaseRpcController> controller, const int64_t& timeout_ns) {
+  controller->Reset();
+  if (timeout_ns >= 0) {
+    controller->set_call_timeout(std::chrono::milliseconds(
+        std::min(static_cast<int64_t>(INT_MAX), TimeUtil::ToMillis(timeout_ns))));
+  }
+}
+
+// explicit instantiations for the linker. Otherwise, you have to #include the .cc file for the
+// templetized
+// class definitions.
+class OpenScannerResponse;
+template class AsyncSingleRequestRpcRetryingCaller<std::shared_ptr<hbase::Result>>;
+template class AsyncSingleRequestRpcRetryingCaller<folly::Unit>;
+template class AsyncSingleRequestRpcRetryingCaller<std::shared_ptr<OpenScannerResponse>>;
+template class AsyncSingleRequestRpcRetryingCaller<bool>;
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-rpc-retrying-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-rpc-retrying-test.cc b/hbase-native-client/src/hbase/client/async-rpc-retrying-test.cc
new file mode 100644
index 0000000..6782d05
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-rpc-retrying-test.cc
@@ -0,0 +1,420 @@
+/*
+ * 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 <folly/Logging.h>
+#include <folly/Memory.h>
+#include <folly/futures/Future.h>
+#include <folly/io/async/EventBase.h>
+#include <folly/io/async/ScopedEventBaseThread.h>
+#include <gmock/gmock.h>
+#include <google/protobuf/stubs/callback.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+
+#include <chrono>
+#include <functional>
+#include <string>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/async-rpc-retrying-caller-factory.h"
+#include "hbase/client/async-rpc-retrying-caller.h"
+#include "hbase/client/client.h"
+#include "hbase/client/connection-configuration.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/keyvalue-codec.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/client/result.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/test-util/test-util.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::AsyncRpcRetryingCallerFactory;
+using hbase::AsyncConnection;
+using hbase::AsyncRegionLocator;
+using hbase::ConnectionConfiguration;
+using hbase::Configuration;
+using hbase::HBaseRpcController;
+using hbase::RegionLocation;
+using hbase::RegionLocateType;
+using hbase::RpcClient;
+using hbase::RequestConverter;
+using hbase::ResponseConverter;
+using hbase::ReqConverter;
+using hbase::RespConverter;
+using hbase::Put;
+using hbase::TimeUtil;
+using hbase::Client;
+using hbase::security::User;
+
+using ::testing::Return;
+using ::testing::_;
+using std::chrono::nanoseconds;
+using std::chrono::milliseconds;
+
+using namespace hbase;
+
+using folly::exception_wrapper;
+
+class AsyncRpcRetryTest : public ::testing::Test {
+ public:
+  static std::unique_ptr<hbase::TestUtil> test_util;
+
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util = std::make_unique<hbase::TestUtil>();
+    test_util->StartMiniCluster(2);
+  }
+};
+std::unique_ptr<hbase::TestUtil> AsyncRpcRetryTest::test_util = nullptr;
+
+class AsyncRegionLocatorBase : public AsyncRegionLocator {
+ public:
+  AsyncRegionLocatorBase() {}
+  explicit AsyncRegionLocatorBase(std::shared_ptr<RegionLocation> region_location)
+      : region_location_(region_location) {}
+  virtual ~AsyncRegionLocatorBase() = default;
+
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(const hbase::pb::TableName &,
+                                                                     const std::string &,
+                                                                     const RegionLocateType,
+                                                                     const int64_t) override {
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    promise.setValue(region_location_);
+    return promise.getFuture();
+  }
+
+  virtual void set_region_location(std::shared_ptr<RegionLocation> region_location) {
+    region_location_ = region_location;
+  }
+
+  void UpdateCachedLocation(const RegionLocation &, const folly::exception_wrapper &) override {}
+
+ protected:
+  std::shared_ptr<RegionLocation> region_location_;
+};
+
+class MockAsyncRegionLocator : public AsyncRegionLocatorBase {
+ public:
+  MockAsyncRegionLocator() : AsyncRegionLocatorBase() {}
+  explicit MockAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockAsyncRegionLocator() {}
+};
+
+class MockWrongRegionAsyncRegionLocator : public AsyncRegionLocatorBase {
+ private:
+  uint32_t tries_ = 0;
+  uint32_t num_fails_ = 0;
+
+ public:
+  explicit MockWrongRegionAsyncRegionLocator(uint32_t num_fails)
+      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
+  explicit MockWrongRegionAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockWrongRegionAsyncRegionLocator() {}
+
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) override {
+    // Fail for num_fails_ times, then delegate to the super class which will give the correct
+    // region location.
+    if (tries_++ > num_fails_) {
+      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
+    }
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    /* set random region name, simulating invalid region */
+    auto result = std::make_shared<RegionLocation>(
+        "whatever-region-name", region_location_->region_info(), region_location_->server_name());
+    promise.setValue(result);
+    return promise.getFuture();
+  }
+};
+
+class MockFailingAsyncRegionLocator : public AsyncRegionLocatorBase {
+ private:
+  uint32_t tries_ = 0;
+  uint32_t num_fails_ = 0;
+
+ public:
+  explicit MockFailingAsyncRegionLocator(uint32_t num_fails)
+      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
+  explicit MockFailingAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
+      : AsyncRegionLocatorBase(region_location) {}
+  virtual ~MockFailingAsyncRegionLocator() {}
+  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) override {
+    // Fail for num_fails_ times, then delegate to the super class which will give the correct
+    // region location.
+    if (tries_++ > num_fails_) {
+      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
+    }
+    folly::Promise<std::shared_ptr<RegionLocation>> promise;
+    promise.setException(std::runtime_error{"Failed to look up region location"});
+    return promise.getFuture();
+  }
+};
+
+class MockAsyncConnection : public AsyncConnection,
+                            public std::enable_shared_from_this<MockAsyncConnection> {
+ public:
+  MockAsyncConnection(std::shared_ptr<ConnectionConfiguration> conn_conf,
+                      std::shared_ptr<folly::HHWheelTimer> retry_timer,
+                      std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                      std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                      std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor,
+                      std::shared_ptr<RpcClient> rpc_client,
+                      std::shared_ptr<AsyncRegionLocator> region_locator)
+      : conn_conf_(conn_conf),
+        retry_timer_(retry_timer),
+        cpu_executor_(cpu_executor),
+        io_executor_(io_executor),
+        retry_executor_(retry_executor),
+        rpc_client_(rpc_client),
+        region_locator_(region_locator) {}
+  ~MockAsyncConnection() {}
+  void Init() {
+    caller_factory_ =
+        std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
+  }
+
+  std::shared_ptr<Configuration> conf() override { return nullptr; }
+  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return conn_conf_; }
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
+    return caller_factory_;
+  }
+  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
+  std::shared_ptr<AsyncRegionLocator> region_locator() override { return region_locator_; }
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
+    return retry_executor_;
+  }
+
+  void Close() override {}
+  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
+    return std::make_shared<HBaseRpcController>();
+  }
+
+ private:
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<ConnectionConfiguration> conn_conf_;
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
+  std::shared_ptr<RpcClient> rpc_client_;
+  std::shared_ptr<AsyncRegionLocator> region_locator_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
+};
+
+template <typename CONN>
+class MockRawAsyncTableImpl {
+ public:
+  explicit MockRawAsyncTableImpl(std::shared_ptr<CONN> conn) : conn_(conn) {}
+  virtual ~MockRawAsyncTableImpl() = default;
+
+  /* implement this in real RawAsyncTableImpl. */
+
+  /* in real RawAsyncTableImpl, this should be private. */
+  folly::Future<std::shared_ptr<hbase::Result>> GetCall(
+      std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
+      std::shared_ptr<RegionLocation> loc, const hbase::Get &get) {
+    hbase::RpcCall<hbase::Request, hbase::Response> rpc_call = [](
+        std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<RegionLocation> loc,
+        std::shared_ptr<HBaseRpcController> controller,
+        std::unique_ptr<hbase::Request> preq) -> folly::Future<std::unique_ptr<hbase::Response>> {
+      VLOG(1) << "entering MockRawAsyncTableImpl#GetCall, calling AsyncCall, loc:"
+              << loc->DebugString();
+      return rpc_client->AsyncCall(loc->server_name().host_name(), loc->server_name().port(),
+                                   std::move(preq), User::defaultUser(), "ClientService");
+    };
+
+    return Call<hbase::Get, hbase::Request, hbase::Response, std::shared_ptr<hbase::Result>>(
+        rpc_client, controller, loc, get, &hbase::RequestConverter::ToGetRequest, rpc_call,
+        &hbase::ResponseConverter::FromGetResponse);
+  }
+
+  /* in real RawAsyncTableImpl, this should be private. */
+  template <typename REQ, typename PREQ, typename PRESP, typename RESP>
+  folly::Future<RESP> Call(std::shared_ptr<hbase::RpcClient> rpc_client,
+                           std::shared_ptr<HBaseRpcController> controller,
+                           std::shared_ptr<RegionLocation> loc, const REQ &req,
+                           ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
+                           hbase::RpcCall<PREQ, PRESP> rpc_call,
+                           RespConverter<RESP, PRESP> resp_converter) {
+    promise_ = std::make_shared<folly::Promise<std::shared_ptr<hbase::Result>>>();
+    auto f = promise_->getFuture();
+    VLOG(1) << "calling rpc_call";
+    rpc_call(rpc_client, loc, controller, std::move(req_converter(req, loc->region_name())))
+        .then([&, this, resp_converter](std::unique_ptr<PRESP> presp) {
+          VLOG(1) << "MockRawAsyncTableImpl#call succeded: ";
+          RESP result = resp_converter(*presp);
+          promise_->setValue(result);
+        })
+        .onError([this](const exception_wrapper &e) {
+          VLOG(1) << "entering MockRawAsyncTableImpl#call, exception: " << e.what();
+          VLOG(1) << "entering MockRawAsyncTableImpl#call, error typeinfo: " << typeid(e).name();
+          promise_->setException(e);
+        });
+    return f;
+  }
+
+ private:
+  std::shared_ptr<CONN> conn_;
+  std::shared_ptr<folly::Promise<std::shared_ptr<hbase::Result>>> promise_;
+};
+
+void runTest(std::shared_ptr<AsyncRegionLocatorBase> region_locator, std::string tableName,
+             uint32_t operation_timeout_millis = 1200000) {
+  AsyncRpcRetryTest::test_util->CreateTable(tableName, "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(tableName);
+  auto row = "test2";
+
+  // Get to be performed on above HBase Table
+  hbase::Get get(row);
+
+  // Create a client
+  Client client(*(AsyncRpcRetryTest::test_util->conf()));
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+
+  table->Put(Put{"test2"}.AddColumn("d", "2", "value2"));
+  table->Put(Put{"test2"}.AddColumn("d", "extra", "value for extra"));
+
+  /* init region location and rpc channel */
+  auto region_location = table->GetRegionLocation(row);
+
+  // auto io_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(4);
+  auto cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
+  auto io_executor_ = client.async_connection()->io_executor();
+  auto retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
+  auto codec = std::make_shared<hbase::KeyValueCodec>();
+  auto rpc_client = std::make_shared<RpcClient>(io_executor_, cpu_executor_, codec,
+                                                AsyncRpcRetryTest::test_util->conf());
+  // auto retry_event_base_ = std::make_shared<folly::ScopedEventBaseThread>(true);
+  std::shared_ptr<folly::HHWheelTimer> retry_timer =
+      folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
+
+  /* init connection configuration */
+  auto connection_conf = std::make_shared<ConnectionConfiguration>(
+      TimeUtil::SecondsToNanos(20),                       // connect_timeout
+      TimeUtil::MillisToNanos(operation_timeout_millis),  // operation_timeout
+      TimeUtil::SecondsToNanos(60),                       // rpc_timeout
+      TimeUtil::MillisToNanos(100),                       // pause
+      5,                                                  // max retries
+      9);                                                 // start log errors count
+
+  /* set region locator */
+  region_locator->set_region_location(region_location);
+
+  /* init hbase client connection */
+  auto conn = std::make_shared<MockAsyncConnection>(connection_conf, retry_timer, cpu_executor_,
+                                                    io_executor_, retry_executor_, rpc_client,
+                                                    region_locator);
+  conn->Init();
+
+  /* init retry caller factory */
+  auto tableImpl = std::make_shared<MockRawAsyncTableImpl<MockAsyncConnection>>(conn);
+
+  /* init request caller builder */
+  auto builder = conn->caller_factory()->Single<std::shared_ptr<hbase::Result>>();
+
+  /* call with retry to get result */
+
+  auto async_caller =
+      builder->table(std::make_shared<hbase::pb::TableName>(tn))
+          ->row(row)
+          ->rpc_timeout(conn->connection_conf()->read_rpc_timeout())
+          ->operation_timeout(conn->connection_conf()->operation_timeout())
+          ->action([=, &get](std::shared_ptr<hbase::HBaseRpcController> controller,
+                             std::shared_ptr<hbase::RegionLocation> loc,
+                             std::shared_ptr<hbase::RpcClient> rpc_client)
+                       -> folly::Future<std::shared_ptr<hbase::Result>> {
+                         return tableImpl->GetCall(rpc_client, controller, loc, get);
+                       })
+          ->Build();
+
+  auto promise = std::make_shared<folly::Promise<std::shared_ptr<hbase::Result>>>();
+
+  auto result = async_caller->Call().get(milliseconds(500000));
+
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ("test2", result->Row());
+  EXPECT_EQ("value2", *(result->Value("d", "2")));
+  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
+
+  retry_timer->destroy();
+  table->Close();
+  client.Close();
+  retry_executor_->stop();
+}
+
+// Test successful case
+TEST_F(AsyncRpcRetryTest, TestGetBasic) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockAsyncRegionLocator>());
+  runTest(region_locator, "table1");
+}
+
+// Tests the RPC failing 3 times, then succeeding
+TEST_F(AsyncRpcRetryTest, TestHandleException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
+  runTest(region_locator, "table2");
+}
+
+// Tests the RPC failing 5 times, throwing an exception
+TEST_F(AsyncRpcRetryTest, TestFailWithException) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockWrongRegionAsyncRegionLocator>(5));
+  EXPECT_ANY_THROW(runTest(region_locator, "table3"));
+}
+
+// Tests the region location lookup failing 3 times, then succeeding
+TEST_F(AsyncRpcRetryTest, TestHandleExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(3));
+  runTest(region_locator, "table4");
+}
+
+// Tests the region location lookup failing 5 times, throwing an exception
+TEST_F(AsyncRpcRetryTest, TestFailWithExceptionFromRegionLocationLookup) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(5));
+  EXPECT_ANY_THROW(runTest(region_locator, "table5"));
+}
+
+// Tests hitting operation timeout, thus not retrying anymore
+TEST_F(AsyncRpcRetryTest, TestFailWithOperationTimeout) {
+  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
+      std::make_shared<MockFailingAsyncRegionLocator>(3));
+  EXPECT_ANY_THROW(runTest(region_locator, "table6", 200));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-scan-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-scan-rpc-retrying-caller.cc b/hbase-native-client/src/hbase/client/async-scan-rpc-retrying-caller.cc
new file mode 100644
index 0000000..2189128
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-scan-rpc-retrying-caller.cc
@@ -0,0 +1,448 @@
+/*
+ * 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 "hbase/client/async-scan-rpc-retrying-caller.h"
+
+namespace hbase {
+
+ScanResumerImpl::ScanResumerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller)
+    : caller_(caller), mutex_() {}
+
+void ScanResumerImpl::Resume() {
+  // just used to fix findbugs warnings. In fact, if resume is called before prepare, then we
+  // just return at the first if condition without loading the resp and numValidResuls field. If
+  // resume is called after suspend, then it is also safe to just reference resp and
+  // numValidResults after the synchronized block as no one will change it anymore.
+  std::shared_ptr<pb::ScanResponse> local_resp;
+  int64_t local_num_complete_rows;
+
+  {
+    std::unique_lock<std::mutex> mlock{mutex_};
+    if (state_ == ScanResumerState::kInitialized) {
+      // user calls this method before we call prepare, so just set the state to
+      // RESUMED, the implementation will just go on.
+      state_ = ScanResumerState::kResumed;
+      return;
+    }
+    if (state_ == ScanResumerState::kResumed) {
+      // already resumed, give up.
+      return;
+    }
+    state_ = ScanResumerState::kResumed;
+    local_resp = resp_;
+    local_num_complete_rows = num_complete_rows_;
+  }
+
+  caller_->CompleteOrNext(local_resp);
+}
+
+bool ScanResumerImpl::Prepare(std::shared_ptr<pb::ScanResponse> resp, int num_complete_rows) {
+  std::unique_lock<std::mutex> mlock(mutex_);
+  if (state_ == ScanResumerState::kResumed) {
+    // user calls resume before we actually suspend the scan, just continue;
+    return false;
+  }
+  state_ = ScanResumerState::kSuspended;
+  resp_ = resp;
+  num_complete_rows_ = num_complete_rows;
+
+  return true;
+}
+
+ScanControllerImpl::ScanControllerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller)
+    : caller_(caller) {}
+
+std::shared_ptr<ScanResumer> ScanControllerImpl::Suspend() {
+  PreCheck();
+  state_ = ScanControllerState::kSuspended;
+  resumer_ = std::make_shared<ScanResumerImpl>(caller_);
+  return resumer_;
+}
+
+void ScanControllerImpl::Terminate() {
+  PreCheck();
+  state_ = ScanControllerState::kTerminated;
+}
+
+// return the current state, and set the state to DESTROYED.
+ScanControllerState ScanControllerImpl::Destroy() {
+  ScanControllerState state = state_;
+  state_ = ScanControllerState::kDestroyed;
+  return state;
+}
+
+void ScanControllerImpl::PreCheck() {
+  CHECK(std::this_thread::get_id() == caller_thread_id_)
+      << "The current thread is" << std::this_thread::get_id() << ", expected thread is "
+      << caller_thread_id_ << ", you should not call this method outside OnNext or OnHeartbeat";
+
+  CHECK(state_ == ScanControllerState::kInitialized) << "Invalid Stopper state "
+                                                     << DebugString(state_);
+}
+
+std::string ScanControllerImpl::DebugString(ScanControllerState state) {
+  switch (state) {
+    case ScanControllerState::kInitialized:
+      return "kInitialized";
+    case ScanControllerState::kSuspended:
+      return "kSuspended";
+    case ScanControllerState::kTerminated:
+      return "kTerminated";
+    case ScanControllerState::kDestroyed:
+      return "kDestroyed";
+    default:
+      return "UNKNOWN";
+  }
+}
+
+std::string ScanControllerImpl::DebugString(ScanResumerState state) {
+  switch (state) {
+    case ScanResumerState::kInitialized:
+      return "kInitialized";
+    case ScanResumerState::kSuspended:
+      return "kSuspended";
+    case ScanResumerState::kResumed:
+      return "kResumed";
+    default:
+      return "UNKNOWN";
+  }
+}
+
+AsyncScanRpcRetryingCaller::AsyncScanRpcRetryingCaller(
+    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
+    std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<Scan> scan, int64_t scanner_id,
+    std::shared_ptr<ScanResultCache> results_cache, std::shared_ptr<RawScanResultConsumer> consumer,
+    std::shared_ptr<RegionLocation> region_location, nanoseconds scanner_lease_timeout_nanos,
+    nanoseconds pause, uint32_t max_retries, nanoseconds scan_timeout_nanos,
+    nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
+    : conn_(conn),
+      retry_timer_(retry_timer),
+      rpc_client_(rpc_client),
+      scan_(scan),
+      scanner_id_(scanner_id),
+      results_cache_(results_cache),
+      consumer_(consumer),
+      region_location_(region_location),
+      scanner_lease_timeout_nanos_(scanner_lease_timeout_nanos),
+      pause_(pause),
+      max_retries_(max_retries),
+      scan_timeout_nanos_(scan_timeout_nanos),
+      rpc_timeout_nanos_(rpc_timeout_nanos),
+      start_log_errors_count_(start_log_errors_count),
+      promise_(std::make_shared<folly::Promise<bool>>()),
+      tries_(1) {
+  controller_ = conn_->CreateRpcController();
+  start_ns_ = TimeUtil::GetNowNanos();
+  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
+  exceptions_ = std::make_shared<std::vector<ThrowableWithExtraContext>>();
+}
+
+folly::Future<bool> AsyncScanRpcRetryingCaller::Start(
+    std::shared_ptr<HBaseRpcController> controller,
+    std::shared_ptr<pb::ScanResponse> open_scan_resp,
+    const std::shared_ptr<CellScanner> cell_scanner) {
+  OnComplete(controller, open_scan_resp, cell_scanner);
+  return promise_->getFuture();
+}
+
+int64_t AsyncScanRpcRetryingCaller::RemainingTimeNs() {
+  return scan_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
+}
+
+void AsyncScanRpcRetryingCaller::OnComplete(std::shared_ptr<HBaseRpcController> controller,
+                                            std::shared_ptr<pb::ScanResponse> resp,
+                                            const std::shared_ptr<CellScanner> cell_scanner) {
+  VLOG(5) << "Scan: OnComplete, scanner_id:" << scanner_id_;
+
+  if (controller->Failed()) {
+    OnError(controller->exception());
+    return;
+  }
+
+  bool is_heartbeat = resp->has_heartbeat_message() && resp->heartbeat_message();
+
+  int64_t num_complete_rows_before = results_cache_->num_complete_rows();
+  try {
+    auto raw_results = ResponseConverter::FromScanResponse(resp, cell_scanner);
+
+    auto results = results_cache_->AddAndGet(raw_results, is_heartbeat);
+
+    auto scan_controller = std::make_shared<ScanControllerImpl>(shared_from_this());
+
+    if (results.size() > 0) {
+      UpdateNextStartRowWhenError(*results[results.size() - 1]);
+      VLOG(5) << "Calling consumer->OnNext()";
+      consumer_->OnNext(results, scan_controller);
+    } else if (is_heartbeat) {
+      consumer_->OnHeartbeat(scan_controller);
+    }
+
+    ScanControllerState state = scan_controller->Destroy();
+    if (state == ScanControllerState::kTerminated) {
+      if (resp->has_more_results_in_region() && !resp->more_results_in_region()) {
+        // we have more results in region but user request to stop the scan, so we need to close the
+        // scanner explicitly.
+        CloseScanner();
+      }
+      CompleteNoMoreResults();
+      return;
+    }
+
+    int64_t num_complete_rows = results_cache_->num_complete_rows() - num_complete_rows_before;
+    if (state == ScanControllerState::kSuspended) {
+      if (scan_controller->resumer()->Prepare(resp, num_complete_rows)) {
+        return;
+      }
+    }
+  } catch (const std::runtime_error& e) {
+    // We can not retry here. The server has responded normally and the call sequence has been
+    // increased so a new scan with the same call sequence will cause an
+    // OutOfOrderScannerNextException. Let the upper layer open a new scanner.
+    LOG(WARNING) << "Received exception in reading the scan response:" << e.what();
+    CompleteWhenError(true);
+    return;
+  }
+
+  CompleteOrNext(resp);
+}
+
+void AsyncScanRpcRetryingCaller::CompleteOrNext(std::shared_ptr<pb::ScanResponse> resp) {
+  VLOG(5) << "Scan: CompleteOrNext, scanner_id" << scanner_id_
+          << ", response:" << resp->ShortDebugString();
+
+  if (resp->has_more_results() && !resp->more_results()) {
+    // RS tells us there is no more data for the whole scan
+    CompleteNoMoreResults();
+    return;
+  }
+  // TODO: Implement Scan::limit(), and check the limit here
+
+  if (resp->has_more_results_in_region() && !resp->more_results_in_region()) {
+    // TODO: check whether Scan is reversed here
+    CompleteWhenNoMoreResultsInRegion();
+    return;
+  }
+  Next();
+}
+
+void AsyncScanRpcRetryingCaller::CompleteExceptionally(bool close_scanner) {
+  VLOG(5) << "Scan: CompleteExceptionally";
+  results_cache_->Clear();
+  if (close_scanner) {
+    CloseScanner();
+  }
+  this->promise_->setException(RetriesExhaustedException(tries_ - 1, exceptions_));
+}
+
+void AsyncScanRpcRetryingCaller::CompleteNoMoreResults() {
+  // In master code, scanners auto-close if we have exhausted the region. It may not be the case
+  // in branch-1 code. If this is backported, make sure that the scanner is closed.
+  VLOG(5) << "Scan: CompleteNoMoreResults, scanner_id:" << scanner_id_;
+  promise_->setValue(false);
+}
+
+void AsyncScanRpcRetryingCaller::CompleteWhenNoMoreResultsInRegion() {
+  VLOG(5) << "Scan: CompleteWhenNoMoreResultsInRegion, scanner_id:" << scanner_id_;
+  // In master code, scanners auto-close if we have exhausted the region. It may not be the case
+  // in branch-1 code. If this is backported, make sure that the scanner is closed.
+  if (NoMoreResultsForScan(*scan_, region_location_->region_info())) {
+    CompleteNoMoreResults();
+  } else {
+    CompleteWithNextStartRow(region_location_->region_info().end_key(), true);
+  }
+}
+
+void AsyncScanRpcRetryingCaller::CompleteWithNextStartRow(std::string row, bool inclusive) {
+  VLOG(5) << "Scan: CompleteWithNextStartRow: region scan is complete, move to next region";
+  scan_->SetStartRow(row);
+  // TODO: set inclusive if it is reverse scans
+  promise_->setValue(true);
+}
+
+void AsyncScanRpcRetryingCaller::UpdateNextStartRowWhenError(const Result& result) {
+  next_start_row_when_error_ = optional<std::string>(result.Row());
+  include_next_start_row_when_error_ = result.Partial();
+}
+
+void AsyncScanRpcRetryingCaller::CompleteWhenError(bool close_scanner) {
+  VLOG(5) << "Scan: CompleteWhenError, scanner_id:" << scanner_id_;
+  results_cache_->Clear();
+  if (close_scanner) {
+    CloseScanner();
+  }
+  if (next_start_row_when_error_) {
+    // TODO: HBASE-17583 adds include start / stop row to the Scan. Once we rebase and implement
+    // those options in Scan , we can start using that here.
+    scan_->SetStartRow(include_next_start_row_when_error_
+                           ? *next_start_row_when_error_
+                           : BytesUtil::CreateClosestRowAfter(*next_start_row_when_error_));
+  }
+  promise_->setValue(true);
+}
+
+void AsyncScanRpcRetryingCaller::OnError(const folly::exception_wrapper& error) {
+  VLOG(5) << "Scan: OnError, scanner_id:" << scanner_id_;
+  if (tries_ > start_log_errors_count_ || VLOG_IS_ON(5)) {
+    LOG(WARNING) << "Call to " << region_location_->server_name().ShortDebugString()
+                 << " for scanner id = " << scanner_id_ << " for "
+                 << region_location_->region_info().ShortDebugString()
+                 << " failed, , tries = " << tries_ << ", maxAttempts = " << max_attempts_
+                 << ", timeout = " << TimeUtil::ToMillis(scan_timeout_nanos_).count()
+                 << " ms, time elapsed = " << TimeUtil::ElapsedMillis(start_ns_) << " ms"
+                 << error.what().toStdString();
+  }
+
+  bool scanner_closed = ExceptionUtil::IsScannerClosed(error);
+  ThrowableWithExtraContext twec(error, TimeUtil::GetNowNanos());
+  exceptions_->push_back(twec);
+  if (tries_ >= max_retries_) {
+    CompleteExceptionally(!scanner_closed);
+    return;
+  }
+
+  int64_t delay_ns;
+  if (scan_timeout_nanos_.count() > 0) {
+    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
+    if (max_delay_ns <= 0) {
+      CompleteExceptionally(!scanner_closed);
+      return;
+    }
+    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1));
+  } else {
+    delay_ns = ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1);
+  }
+
+  if (scanner_closed) {
+    CompleteWhenError(false);
+    return;
+  }
+
+  if (ExceptionUtil::IsScannerOutOfOrder(error)) {
+    CompleteWhenError(true);
+    return;
+  }
+  if (!ExceptionUtil::ShouldRetry(error)) {
+    CompleteExceptionally(true);
+    return;
+  }
+  tries_++;
+
+  auto self(shared_from_this());
+  conn_->retry_executor()->add([&]() {
+    retry_timer_->scheduleTimeoutFn(
+        [self]() { self->conn_->cpu_executor()->add([&]() { self->Call(); }); },
+        std::chrono::milliseconds(TimeUtil::ToMillis(delay_ns)));
+  });
+}
+
+bool AsyncScanRpcRetryingCaller::NoMoreResultsForScan(const Scan& scan,
+                                                      const pb::RegionInfo& info) {
+  if (BytesUtil::IsEmptyStopRow(info.end_key())) {
+    return true;
+  }
+  if (BytesUtil::IsEmptyStopRow(scan.StopRow())) {
+    return false;
+  }
+  int32_t c = BytesUtil::CompareTo(info.end_key(), scan.StopRow());
+  // 1. if our stop row is less than the endKey of the region
+  // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
+  // for scan.
+  return c > 0 ||
+         (c == 0 /* && !scan.IncludeStopRow()*/);  // TODO: Scans always exclude StopRow for now.
+}
+
+void AsyncScanRpcRetryingCaller::Next() {
+  VLOG(5) << "Scan: Next";
+  next_call_seq_++;
+  tries_ = 1;
+  exceptions_->clear();
+  start_ns_ = TimeUtil::GetNowNanos();
+  Call();
+}
+
+void AsyncScanRpcRetryingCaller::Call() {
+  VLOG(5) << "Scan: Call";
+  auto self(shared_from_this());
+  // As we have a call sequence for scan, it is useless to have a different rpc timeout which is
+  // less than the scan timeout. If the server does not respond in time(usually this will not
+  // happen as we have heartbeat now), we will get an OutOfOrderScannerNextException when
+  // resending the next request and the only way to fix this is to close the scanner and open a
+  // new one.
+  int64_t call_timeout_nanos;
+  if (scan_timeout_nanos_.count() > 0) {
+    int64_t remaining_nanos = scan_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
+    if (remaining_nanos <= 0) {
+      CompleteExceptionally(true);
+      return;
+    }
+    call_timeout_nanos = remaining_nanos;
+  } else {
+    call_timeout_nanos = 0L;
+  }
+
+  ResetController(controller_, call_timeout_nanos);
+
+  auto req =
+      RequestConverter::ToScanRequest(scanner_id_, scan_->Caching(), false, next_call_seq_, false);
+
+  // do the RPC call
+  rpc_client_
+      ->AsyncCall(region_location_->server_name().host_name(),
+                  region_location_->server_name().port(), std::move(req),
+                  security::User::defaultUser(), "ClientService")
+      .via(conn_->cpu_executor().get())
+      .then([self, this](const std::unique_ptr<Response>& resp) {
+        auto scan_resp = std::static_pointer_cast<pb::ScanResponse>(resp->resp_msg());
+        return OnComplete(controller_, scan_resp, resp->cell_scanner());
+      })
+      .onError([self, this](const folly::exception_wrapper& e) { OnError(e); });
+}
+
+void AsyncScanRpcRetryingCaller::CloseScanner() {
+  auto self(shared_from_this());
+  ResetController(controller_, rpc_timeout_nanos_.count());
+
+  VLOG(5) << "Closing scanner with scanner_id:" << folly::to<std::string>(scanner_id_);
+
+  // Do a close scanner RPC. Fire and forget.
+  auto req = RequestConverter::ToScanRequest(scanner_id_, 0, true);
+  rpc_client_
+      ->AsyncCall(region_location_->server_name().host_name(),
+                  region_location_->server_name().port(), std::move(req),
+                  security::User::defaultUser(), "ClientService")
+      .onError([self, this](const folly::exception_wrapper& e) -> std::unique_ptr<Response> {
+        LOG(WARNING) << "Call to " + region_location_->server_name().ShortDebugString() +
+                            " for closing scanner_id = " + folly::to<std::string>(scanner_id_) +
+                            " for " + region_location_->region_info().ShortDebugString() +
+                            " failed, ignore, probably already closed. Exception:" +
+                            e.what().toStdString();
+        return nullptr;
+      });
+}
+
+void AsyncScanRpcRetryingCaller::ResetController(std::shared_ptr<HBaseRpcController> controller,
+                                                 const int64_t& timeout_nanos) {
+  controller->Reset();
+  if (timeout_nanos >= 0) {
+    controller->set_call_timeout(
+        milliseconds(std::min(static_cast<int64_t>(INT_MAX), TimeUtil::ToMillis(timeout_nanos))));
+  }
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-table-result-scanner.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-table-result-scanner.cc b/hbase-native-client/src/hbase/client/async-table-result-scanner.cc
new file mode 100644
index 0000000..3812b3a
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-table-result-scanner.cc
@@ -0,0 +1,161 @@
+/*
+ * 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 "hbase/client/async-table-result-scanner.h"
+
+#include <vector>
+
+namespace hbase {
+AsyncTableResultScanner::AsyncTableResultScanner(int64_t max_cache_size)
+    : max_cache_size_(max_cache_size) {
+  closed_ = false;
+  cache_size_ = 0;
+}
+
+AsyncTableResultScanner::~AsyncTableResultScanner() { Close(); }
+
+void AsyncTableResultScanner::Close() {
+  std::unique_lock<std::mutex> mlock(mutex_);
+  closed_ = true;
+  while (!queue_.empty()) {
+    queue_.pop();
+  }
+  cache_size_ = 0;
+  if (resumer_ != nullptr) {
+    resumer_->Resume();
+  }
+  cond_.notify_all();
+}
+
+std::shared_ptr<Result> AsyncTableResultScanner::Next() {
+  VLOG(5) << "AsyncTableResultScanner: Next()";
+
+  std::shared_ptr<Result> result = nullptr;
+  std::shared_ptr<ScanResumer> local_resumer = nullptr;
+  {
+    std::unique_lock<std::mutex> mlock(mutex_);
+    while (queue_.empty()) {
+      if (closed_) {
+        return nullptr;
+      }
+      if (error_) {
+        throw error_;
+      }
+      cond_.wait(mlock);
+    }
+    result = queue_.front();
+    queue_.pop();
+
+    cache_size_ -= EstimatedSizeWithSharedPtr(result);
+    if (resumer_ != nullptr && cache_size_ <= max_cache_size_ / 2) {
+      VLOG(1) << std::this_thread::get_id() << " resume scan prefetching";
+      local_resumer = resumer_;
+      resumer_ = nullptr;
+    }
+  }
+
+  // Need to call ScanResumer::Resume() outside of the scope of the mutex. The reason is that
+  // folly/wangle event loop might end up running the attached logic(.then()) at the Scan RPC
+  // in the same event thread before returning from the previous call. This seems like the
+  // wrong thing to do(â„¢), but we cannot fix that now. Since the call back can end up calling
+  // this::OnNext(), we should unlock the mutex.
+  if (local_resumer != nullptr) {
+    local_resumer->Resume();
+  }
+  return result;
+}
+
+void AsyncTableResultScanner::AddToCache(const std::vector<std::shared_ptr<Result>> &results) {
+  VLOG(5) << "AsyncTableResultScanner: AddToCache()";
+  for (const auto r : results) {
+    queue_.push(r);
+    cache_size_ += EstimatedSizeWithSharedPtr(r);
+  }
+}
+
+template <typename T>
+inline size_t AsyncTableResultScanner::EstimatedSizeWithSharedPtr(std::shared_ptr<T> t) {
+  return t->EstimatedSize() + sizeof(std::shared_ptr<T>);
+}
+
+void AsyncTableResultScanner::OnNext(const std::vector<std::shared_ptr<Result>> &results,
+                                     std::shared_ptr<ScanController> controller) {
+  VLOG(5) << "AsyncTableResultScanner: OnNext()";
+  {
+    std::unique_lock<std::mutex> mlock(mutex_);
+    if (closed_) {
+      controller->Terminate();
+      return;
+    }
+    AddToCache(results);
+
+    if (cache_size_ >= max_cache_size_) {
+      StopPrefetch(controller);
+    }
+  }
+  cond_.notify_all();
+}
+
+void AsyncTableResultScanner::StopPrefetch(std::shared_ptr<ScanController> controller) {
+  VLOG(1) << std::this_thread::get_id()
+          << ": stop prefetching when scanning as the cache size " +
+                 folly::to<std::string>(cache_size_) + " is greater than the max_cache_size " +
+                 folly::to<std::string>(max_cache_size_);
+
+  resumer_ = controller->Suspend();
+  num_prefetch_stopped_++;
+}
+
+/**
+ * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
+ * onNext.
+ * <p>
+ * This method give you a chance to terminate a slow scan operation.
+ * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
+ *          instance is only valid within the scope of onHeartbeat method. You can only call its
+ *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
+ */
+void AsyncTableResultScanner::OnHeartbeat(std::shared_ptr<ScanController> controller) {
+  std::unique_lock<std::mutex> mlock(mutex_);
+  if (closed_) {
+    controller->Terminate();
+  }
+}
+
+/**
+ * Indicate that we hit an unrecoverable error and the scan operation is terminated.
+ * <p>
+ * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
+ */
+void AsyncTableResultScanner::OnError(const folly::exception_wrapper &error) {
+  LOG(WARNING) << "Scanner received error" << error.what();
+  std::unique_lock<std::mutex> mlock(mutex_);
+  error_ = error;
+  cond_.notify_all();
+}
+
+/**
+ * Indicate that the scan operation is completed normally.
+ */
+void AsyncTableResultScanner::OnComplete() {
+  std::unique_lock<std::mutex> mlock(mutex_);
+  closed_ = true;
+  cond_.notify_all();
+}
+}  // namespace hbase


[02/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/SecureBulkLoad.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/SecureBulkLoad.proto b/hbase-native-client/src/hbase/if/SecureBulkLoad.proto
new file mode 100644
index 0000000..814735b
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/SecureBulkLoad.proto
@@ -0,0 +1,72 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "SecureBulkLoadProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import 'HBase.proto';
+import 'Client.proto';
+
+message SecureBulkLoadHFilesRequest {
+  repeated BulkLoadHFileRequest.FamilyPath family_path = 1;
+  optional bool assign_seq_num = 2;
+  required DelegationToken fs_token = 3;
+  required string bulk_token = 4;
+}
+
+message SecureBulkLoadHFilesResponse {
+  required bool loaded = 1;
+}
+
+message DelegationToken {
+  optional bytes identifier = 1;
+  optional bytes password = 2;
+  optional string kind = 3;
+  optional string service = 4;
+}
+
+message PrepareBulkLoadRequest {
+  required TableName table_name = 1;
+}
+
+message PrepareBulkLoadResponse {
+  required string bulk_token = 1;
+}
+
+message CleanupBulkLoadRequest {
+  required string bulk_token = 1;
+
+}
+
+message CleanupBulkLoadResponse {
+}
+
+service SecureBulkLoadService {
+    rpc PrepareBulkLoad(PrepareBulkLoadRequest)
+      returns (PrepareBulkLoadResponse);
+
+    rpc SecureBulkLoadHFiles(SecureBulkLoadHFilesRequest)
+      returns (SecureBulkLoadHFilesResponse);
+
+    rpc CleanupBulkLoad(CleanupBulkLoadRequest)
+      returns (CleanupBulkLoadResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Snapshot.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Snapshot.proto b/hbase-native-client/src/hbase/if/Snapshot.proto
new file mode 100644
index 0000000..ae1a1e6
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Snapshot.proto
@@ -0,0 +1,66 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "SnapshotProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "FS.proto";
+import "HBase.proto";
+
+message SnapshotFileInfo {
+  enum Type {
+    HFILE = 1;
+    WAL = 2;
+  }
+
+  required Type type = 1;
+
+  optional string hfile = 3;
+
+  optional string wal_server = 4;
+  optional string wal_name = 5;
+}
+
+message SnapshotRegionManifest {
+  optional int32 version = 1;
+
+  required RegionInfo region_info = 2;
+  repeated FamilyFiles family_files = 3;
+
+  message StoreFile {
+    required string name = 1;
+    optional Reference reference = 2;
+
+    // TODO: Add checksums or other fields to verify the file
+    optional uint64 file_size = 3;
+  }
+
+  message FamilyFiles {
+    required bytes family_name = 1;
+    repeated StoreFile store_files = 2;
+  }
+}
+
+message SnapshotDataManifest {
+  required TableSchema table_schema = 1;
+  repeated SnapshotRegionManifest region_manifests = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Tracing.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Tracing.proto b/hbase-native-client/src/hbase/if/Tracing.proto
new file mode 100644
index 0000000..5a64cfc
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Tracing.proto
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "TracingProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+//Used to pass through the information necessary to continue
+//a trace after an RPC is made. All we need is the traceid 
+//(so we know the overarching trace this message is a part of), and
+//the id of the current span when this message was sent, so we know 
+//what span caused the new span we will create when this message is received.
+message RPCTInfo {
+  optional int64 trace_id = 1;
+  optional int64 parent_id = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/VisibilityLabels.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/VisibilityLabels.proto b/hbase-native-client/src/hbase/if/VisibilityLabels.proto
new file mode 100644
index 0000000..d2dc44d
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/VisibilityLabels.proto
@@ -0,0 +1,83 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "VisibilityLabelsProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "Client.proto";
+
+message VisibilityLabelsRequest {
+  repeated VisibilityLabel visLabel = 1;
+}
+
+message VisibilityLabel {
+  required bytes label = 1;
+  optional uint32 ordinal = 2;
+}
+
+message VisibilityLabelsResponse {
+  repeated RegionActionResult result = 1; 
+}
+
+message SetAuthsRequest {
+  required bytes user = 1;
+  repeated bytes auth = 2;
+}
+
+message UserAuthorizations {
+  required bytes user = 1;
+  repeated uint32 auth = 2;
+}
+
+message MultiUserAuthorizations {
+  repeated UserAuthorizations userAuths = 1;
+}
+
+message GetAuthsRequest {
+  required bytes user = 1;
+}
+
+message GetAuthsResponse {
+  required bytes user = 1;
+  repeated bytes auth = 2;
+}
+
+message ListLabelsRequest {
+  optional string regex = 1;
+}
+
+message ListLabelsResponse {
+  repeated bytes label = 1;
+}
+
+service VisibilityLabelsService {
+  rpc addLabels(VisibilityLabelsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc setAuths(SetAuthsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc clearAuths(SetAuthsRequest)
+    returns (VisibilityLabelsResponse);
+  rpc getAuths(GetAuthsRequest)
+    returns (GetAuthsResponse);
+  rpc listLabels(ListLabelsRequest)
+    returns (ListLabelsResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/WAL.proto b/hbase-native-client/src/hbase/if/WAL.proto
new file mode 100644
index 0000000..2061b22
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/WAL.proto
@@ -0,0 +1,173 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "WALProtos";
+option java_generic_services = false;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Client.proto";
+
+message WALHeader {
+  optional bool has_compression = 1;
+  optional bytes encryption_key = 2;
+  optional bool has_tag_compression = 3;
+  optional string writer_cls_name = 4;
+  optional string cell_codec_cls_name = 5;
+}
+
+/*
+ * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
+ * for some KVs
+ */
+message WALKey {
+  required bytes encoded_region_name = 1;
+  required bytes table_name = 2;
+  required uint64 log_sequence_number = 3;
+  required uint64 write_time = 4;
+  /*
+  This parameter is deprecated in favor of clusters which
+  contains the list of clusters that have consumed the change.
+  It is retained so that the log created by earlier releases (0.94)
+  can be read by the newer releases.
+  */
+  optional UUID cluster_id = 5 [deprecated=true];
+
+  repeated FamilyScope scopes = 6;
+  optional uint32 following_kv_count = 7;
+
+  /*
+  This field contains the list of clusters that have
+  consumed the change
+  */
+  repeated UUID cluster_ids = 8;
+
+  optional uint64 nonceGroup = 9;
+  optional uint64 nonce = 10;
+  optional uint64 orig_sequence_number = 11;
+
+/*
+  optional CustomEntryType custom_entry_type = 9;
+
+  enum CustomEntryType {
+    COMPACTION = 0;
+  }
+*/
+}
+
+enum ScopeType {
+  REPLICATION_SCOPE_LOCAL = 0;
+  REPLICATION_SCOPE_GLOBAL = 1;
+}
+
+message FamilyScope {
+  required bytes family = 1;
+  required ScopeType scope_type = 2;
+}
+
+/**
+ * Custom WAL entries
+ */
+
+/**
+ * Special WAL entry to hold all related to a compaction.
+ * Written to WAL before completing compaction.  There is
+ * sufficient info in the below message to complete later
+ * the * compaction should we fail the WAL write.
+ */
+message CompactionDescriptor {
+  required bytes table_name = 1; // TODO: WALKey already stores these, might remove
+  required bytes encoded_region_name = 2;
+  required bytes family_name = 3;
+  repeated string compaction_input = 4; // relative to store dir
+  repeated string compaction_output = 5;
+  required string store_home_dir = 6; // relative to region dir
+  optional bytes  region_name = 7; // full region name
+}
+
+/**
+ * Special WAL entry to hold all related to a flush.
+ */
+message FlushDescriptor {
+  enum FlushAction {
+    START_FLUSH = 0;
+    COMMIT_FLUSH = 1;
+    ABORT_FLUSH = 2;
+    CANNOT_FLUSH = 3; // marker for indicating that a flush has been requested but cannot complete
+  }
+
+  message StoreFlushDescriptor {
+    required bytes family_name = 1;
+    required string store_home_dir = 2; //relative to region dir
+    repeated string flush_output = 3; // relative to store dir (if this is a COMMIT_FLUSH)
+  }
+
+  required FlushAction action = 1;
+  required bytes table_name = 2;
+  required bytes encoded_region_name = 3;
+  optional uint64 flush_sequence_number = 4;
+  repeated StoreFlushDescriptor store_flushes = 5;
+  optional bytes  region_name = 6; // full region name
+}
+
+message StoreDescriptor {
+  required bytes family_name = 1;
+  required string store_home_dir = 2; //relative to region dir
+  repeated string store_file = 3; // relative to store dir
+  optional uint64 store_file_size_bytes = 4; // size of store file
+}
+
+/**
+ * Special WAL entry used for writing bulk load events to WAL
+ */
+message BulkLoadDescriptor {
+  required TableName table_name = 1;
+  required bytes encoded_region_name = 2;
+  repeated StoreDescriptor stores = 3;
+  required int64 bulkload_seq_num = 4;
+}
+
+/**
+ * Special WAL entry to hold all related to a region event (open/close).
+ */
+message RegionEventDescriptor {
+  enum EventType {
+    REGION_OPEN = 0;
+    REGION_CLOSE = 1;
+  }
+
+  required EventType event_type = 1;
+  required bytes table_name = 2;
+  required bytes encoded_region_name = 3;
+  optional uint64 log_sequence_number = 4;
+  repeated StoreDescriptor stores = 5;
+  optional ServerName server = 6;  // Server who opened the region
+  optional bytes  region_name = 7; // full region name
+}
+
+/**
+ * A trailer that is appended to the end of a properly closed WAL file.
+ * If missing, this is either a legacy or a corrupted WAL file.
+ * N.B. This trailer currently doesn't contain any information and we
+ * purposefully don't expose it in the WAL APIs. It's for future growth.
+ */
+message WALTrailer {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/ZooKeeper.proto b/hbase-native-client/src/hbase/if/ZooKeeper.proto
new file mode 100644
index 0000000..41c0e0e
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/ZooKeeper.proto
@@ -0,0 +1,176 @@
+/**
+ * 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.
+ */
+
+// ZNode data in hbase are serialized protobufs with a four byte
+// 'magic' 'PBUF' prefix.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ZooKeeperProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "ClusterStatus.proto";
+
+/**
+ * Content of the meta-region-server znode.
+ */
+message MetaRegionServer {
+  // The ServerName hosting the meta region currently, or destination server,
+  // if meta region is in transition.
+  required ServerName server = 1;
+  // The major version of the rpc the server speaks.  This is used so that
+  // clients connecting to the cluster can have prior knowledge of what version
+  // to send to a RegionServer.  AsyncHBase will use this to detect versions.
+  optional uint32 rpc_version = 2;
+
+  // State of the region transition. OPEN means fully operational 'hbase:meta'
+  optional RegionState.State state = 3;
+}
+
+/**
+ * Content of the master znode.
+ */
+message Master {
+  // The ServerName of the current Master
+  required ServerName master = 1;
+  // Major RPC version so that clients can know what version the master can accept.
+  optional uint32 rpc_version = 2;
+  optional uint32 info_port = 3;
+}
+
+/**
+ * Content of the '/hbase/running', cluster state, znode.
+ */
+message ClusterUp {
+  // If this znode is present, cluster is up.  Currently
+  // the data is cluster start_date.
+  required string start_date = 1;
+}
+
+/**
+ * WAL SplitLog directory znodes have this for content.  Used doing distributed
+ * WAL splitting.  Holds current state and name of server that originated split.
+ */
+message SplitLogTask {
+  enum State {
+    UNASSIGNED = 0;
+    OWNED = 1;
+    RESIGNED = 2;
+    DONE = 3;
+    ERR = 4;
+  }
+  enum RecoveryMode {
+    UNKNOWN = 0;
+    LOG_SPLITTING = 1;
+    LOG_REPLAY = 2;
+  }
+  required State state = 1;
+  required ServerName server_name = 2;
+  optional RecoveryMode mode = 3 [default = UNKNOWN];
+}
+
+/**
+ * The znode that holds state of table.
+ * Deprected, table state is stored in table descriptor on HDFS.
+ */
+message DeprecatedTableState {
+  // Table's current state
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+    DISABLING = 2;
+    ENABLING = 3;
+  }
+  // This is the table's state.  If no znode for a table,
+  // its state is presumed enabled.  See o.a.h.h.zookeeper.ZKTable class
+  // for more.
+  required State state = 1 [default = ENABLED];
+}
+
+message TableCF {
+  optional TableName table_name = 1;
+  repeated bytes families = 2;
+}
+
+/**
+ * Used by replication. Holds a replication peer key.
+ */
+message ReplicationPeer {
+  // clusterkey is the concatenation of the slave cluster's
+  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+  required string clusterkey = 1;
+  optional string replicationEndpointImpl = 2;
+  repeated BytesBytesPair data = 3;
+  repeated NameStringPair configuration = 4;
+  repeated TableCF table_cfs = 5;
+}
+
+/**
+ * Used by replication. Holds whether enabled or disabled
+ */
+message ReplicationState {
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+  }
+  required State state = 1;
+}
+
+/**
+ * Used by replication. Holds the current position in an WAL file.
+ */
+message ReplicationHLogPosition {
+  required int64 position = 1;
+}
+
+/**
+ * Used by replication. Used to lock a region server during failover.
+ */
+message ReplicationLock {
+  required string lock_owner = 1;
+}
+
+/**
+ * Metadata associated with a table lock in zookeeper
+ */
+message TableLock {
+  optional TableName table_name = 1;
+  optional ServerName lock_owner = 2;
+  optional int64 thread_id = 3;
+  optional bool is_shared = 4;
+  optional string purpose = 5;
+  optional int64 create_time = 6;
+}
+
+/**
+ * State of the switch.
+ */
+message SwitchState {
+  optional bool enabled = 1;
+}
+
+/**
+ * State for split and merge, used in hbck
+ */
+message SplitAndMergeState {
+  optional bool split_enabled = 1;
+  optional bool merge_enabled = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/test.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/test.proto b/hbase-native-client/src/hbase/if/test.proto
new file mode 100644
index 0000000..72b68e9
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/test.proto
@@ -0,0 +1,43 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
+option java_outer_classname = "TestProtos";
+option java_generate_equals_and_hash = true;
+
+message EmptyRequestProto {
+}
+
+message EmptyResponseProto {
+}
+
+message EchoRequestProto {
+  required string message = 1;
+}
+
+message EchoResponseProto {
+  required string message = 1;
+}
+
+message PauseRequestProto {
+  required uint32 ms = 1;
+}
+
+message AddrResponseProto {
+  required string addr = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/test_rpc_service.proto b/hbase-native-client/src/hbase/if/test_rpc_service.proto
new file mode 100644
index 0000000..2730403
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/test_rpc_service.proto
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
+option java_outer_classname = "TestRpcServiceProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "test.proto";
+
+
+/**
+ * A protobuf service for use in tests
+ */
+service TestProtobufRpcProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo(EchoRequestProto) returns (EchoResponseProto);
+  rpc error(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc pause(PauseRequestProto) returns (EmptyResponseProto);
+  rpc addr(EmptyRequestProto) returns (AddrResponseProto);
+  rpc socketNotOpen(EmptyRequestProto) returns (EmptyResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/security/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/security/BUCK b/hbase-native-client/src/hbase/security/BUCK
new file mode 100644
index 0000000..c329f30
--- /dev/null
+++ b/hbase-native-client/src/hbase/security/BUCK
@@ -0,0 +1,27 @@
+##
+# 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.
+
+# This is the library dealing with a single connection
+# to a single server.
+cxx_library(
+    name="security",
+    srcs=[],
+    deps=["//include/hbase/security:security", "//src/hbase/client:conf"],
+    compiler_flags=['-Weffc++'],
+    visibility=[
+        'PUBLIC',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/BUCK b/hbase-native-client/src/hbase/serde/BUCK
new file mode 100644
index 0000000..6b39e0b
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/BUCK
@@ -0,0 +1,86 @@
+##
+# 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.
+
+cxx_library(
+    name="serde",
+    srcs=[
+        "rpc-serde.cc",
+        "zk.cc",
+    ],
+    deps=[
+        "//include/hbase/serde:serde", "//src/hbase/if:if", "//third-party:folly", "//src/hbase/utils:utils", "//src/hbase/security:security"
+    ],
+    tests=[
+        ":client-deserializer-test",
+        ":client-serializer-test",
+        ":server-name-test",
+        ":table-name-test",
+        ":zk-deserializer-test",
+        ":region-info-deserializer-test",
+    ],
+    compiler_flags=['-Weffc++'],
+    visibility=[
+        'PUBLIC',
+    ],)
+cxx_test(
+    name="table-name-test",
+    srcs=[
+        "table-name-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)
+cxx_test(
+    name="server-name-test",
+    srcs=[
+        "server-name-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)
+cxx_test(
+    name="client-serializer-test",
+    srcs=[
+        "client-serializer-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)
+cxx_test(
+    name="client-deserializer-test",
+    srcs=[
+        "client-deserializer-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)
+cxx_test(
+    name="zk-deserializer-test",
+    srcs=[
+        "zk-deserializer-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)
+cxx_test(
+    name="region-info-deserializer-test",
+    srcs=[
+        "region-info-deserializer-test.cc",
+    ],
+    deps=[
+        ":serde",
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/client-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/client-deserializer-test.cc b/hbase-native-client/src/hbase/serde/client-deserializer-test.cc
new file mode 100644
index 0000000..3e4c42b
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/client-deserializer-test.cc
@@ -0,0 +1,64 @@
+/*
+ * 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 <folly/io/IOBuf.h>
+#include <gtest/gtest.h>
+
+#include "hbase/if/Client.pb.h"
+#include "hbase/serde/rpc-serde.h"
+
+using namespace hbase;
+using folly::IOBuf;
+using hbase::pb::GetRequest;
+using hbase::pb::RegionSpecifier;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+
+TEST(TestRpcSerde, TestReturnFalseOnNullPtr) {
+  RpcSerde deser{nullptr};
+  ASSERT_LT(deser.ParseDelimited(nullptr, nullptr), 0);
+}
+
+TEST(TestRpcSerde, TestReturnFalseOnBadInput) {
+  RpcSerde deser{nullptr};
+  auto buf = IOBuf::copyBuffer("test");
+  GetRequest gr;
+
+  ASSERT_LT(deser.ParseDelimited(buf.get(), &gr), 0);
+}
+
+TEST(TestRpcSerde, TestGoodGetRequestFullRoundTrip) {
+  GetRequest in;
+  RpcSerde ser{nullptr};
+  RpcSerde deser{nullptr};
+
+  // fill up the GetRequest.
+  in.mutable_region()->set_value("test_region_id");
+  in.mutable_region()->set_type(
+      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_ENCODED_REGION_NAME);
+  in.mutable_get()->set_row("test_row");
+
+  // Create the buffer
+  auto buf = ser.SerializeDelimited(in);
+
+  GetRequest out;
+
+  int used_bytes = deser.ParseDelimited(buf.get(), &out);
+
+  ASSERT_GT(used_bytes, 0);
+  ASSERT_EQ(used_bytes, buf->length());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/client-serializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/client-serializer-test.cc b/hbase-native-client/src/hbase/serde/client-serializer-test.cc
new file mode 100644
index 0000000..8279caa
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/client-serializer-test.cc
@@ -0,0 +1,75 @@
+/*
+ * 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 <gtest/gtest.h>
+
+#include <folly/io/Cursor.h>
+
+#include <string>
+
+#include "hbase/if/HBase.pb.h"
+#include "hbase/if/RPC.pb.h"
+#include "hbase/serde/rpc-serde.h"
+
+using namespace hbase;
+using namespace hbase::pb;
+using namespace folly;
+using namespace folly::io;
+
+TEST(RpcSerdeTest, PreambleIncludesHBas) {
+  RpcSerde ser{nullptr};
+  auto buf = ser.Preamble(false);
+  const char *p = reinterpret_cast<const char *>(buf->data());
+  // Take the first for chars and make sure they are the
+  // magic string
+  EXPECT_EQ("HBas", std::string(p, 4));
+
+  EXPECT_EQ(6, buf->computeChainDataLength());
+}
+
+TEST(RpcSerdeTest, PreambleIncludesVersion) {
+  RpcSerde ser{nullptr};
+  auto buf = ser.Preamble(false);
+  EXPECT_EQ(0, static_cast<const uint8_t *>(buf->data())[4]);
+  EXPECT_EQ(80, static_cast<const uint8_t *>(buf->data())[5]);
+}
+
+TEST(RpcSerdeTest, TestHeaderLengthPrefixed) {
+  RpcSerde ser{nullptr};
+  auto header = ser.Header("elliott");
+
+  // The header should be prefixed by 4 bytes of length.
+  EXPECT_EQ(4, header->length());
+  EXPECT_TRUE(header->length() < header->computeChainDataLength());
+  EXPECT_TRUE(header->isChained());
+
+  // Now make sure the length is correct.
+  Cursor cursor(header.get());
+  auto prefixed_len = cursor.readBE<uint32_t>();
+  EXPECT_EQ(prefixed_len, header->next()->length());
+}
+
+TEST(RpcSerdeTest, TestHeaderDecode) {
+  RpcSerde ser{nullptr};
+  auto buf = ser.Header("elliott");
+  auto header_buf = buf->next();
+  ConnectionHeader h;
+
+  EXPECT_TRUE(h.ParseFromArray(header_buf->data(), header_buf->length()));
+  EXPECT_EQ("elliott", h.user_info().effective_user());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/region-info-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/region-info-deserializer-test.cc b/hbase-native-client/src/hbase/serde/region-info-deserializer-test.cc
new file mode 100644
index 0000000..84219d8
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/region-info-deserializer-test.cc
@@ -0,0 +1,53 @@
+/*
+ * 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 "hbase/serde/region-info.h"
+
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/table-name.h"
+
+using std::string;
+using hbase::pb::RegionInfo;
+using hbase::pb::TableName;
+
+TEST(TestRegionInfoDesializer, TestDeserialize) {
+  string ns{"test_ns"};
+  string tn{"table_name"};
+  string start_row{"AAAAAA"};
+  string stop_row{"BBBBBBBBBBBB"};
+  uint64_t region_id = 2345678;
+
+  RegionInfo ri_out;
+  ri_out.set_region_id(region_id);
+  ri_out.mutable_table_name()->set_namespace_(ns);
+  ri_out.mutable_table_name()->set_qualifier(tn);
+  ri_out.set_start_key(start_row);
+  ri_out.set_end_key(stop_row);
+
+  string header{"PBUF"};
+  string ser = header + ri_out.SerializeAsString();
+
+  auto out = folly::to<RegionInfo>(ser);
+
+  EXPECT_EQ(region_id, out.region_id());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/rpc-serde.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/rpc-serde.cc b/hbase-native-client/src/hbase/serde/rpc-serde.cc
new file mode 100644
index 0000000..5f2920d
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/rpc-serde.cc
@@ -0,0 +1,261 @@
+/*
+ * 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 <folly/Conv.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>
+#include <google/protobuf/message.h>
+#include <boost/algorithm/string.hpp>
+
+#include <utility>
+
+#include "hbase/if/RPC.pb.h"
+#include "hbase/serde/rpc-serde.h"
+#include "hbase/utils/version.h"
+
+using folly::IOBuf;
+using folly::io::RWPrivateCursor;
+using google::protobuf::Message;
+using google::protobuf::io::ArrayInputStream;
+using google::protobuf::io::ArrayOutputStream;
+using google::protobuf::io::CodedInputStream;
+using google::protobuf::io::CodedOutputStream;
+using google::protobuf::io::ZeroCopyOutputStream;
+
+using namespace hbase::pb;
+
+namespace hbase {
+
+static const std::string PREAMBLE = "HBas";
+static const std::string INTERFACE = "ClientService";
+static const uint8_t RPC_VERSION = 0;
+static const uint8_t DEFAULT_AUTH_TYPE = 80;
+static const uint8_t KERBEROS_AUTH_TYPE = 81;
+
+int RpcSerde::ParseDelimited(const IOBuf *buf, Message *msg) {
+  if (buf == nullptr || msg == nullptr) {
+    return -2;
+  }
+
+  DCHECK(!buf->isChained());
+
+  ArrayInputStream ais{buf->data(), static_cast<int>(buf->length())};
+  CodedInputStream coded_stream{&ais};
+
+  uint32_t msg_size;
+
+  // Try and read the varint.
+  if (coded_stream.ReadVarint32(&msg_size) == false) {
+    FB_LOG_EVERY_MS(ERROR, 1000) << "Unable to read a var uint32_t";
+    return -3;
+  }
+
+  coded_stream.PushLimit(msg_size);
+  // Parse the message.
+  if (msg->MergeFromCodedStream(&coded_stream) == false) {
+    FB_LOG_EVERY_MS(ERROR, 1000) << "Unable to read a protobuf message from data.";
+    return -4;
+  }
+
+  // Make sure all the data was consumed.
+  if (coded_stream.ConsumedEntireMessage() == false) {
+    FB_LOG_EVERY_MS(ERROR, 1000) << "Orphaned data left after reading protobuf message";
+    return -5;
+  }
+
+  return coded_stream.CurrentPosition();
+}
+
+RpcSerde::RpcSerde() {}
+
+RpcSerde::RpcSerde(std::shared_ptr<Codec> codec) : codec_(codec) {}
+
+std::unique_ptr<IOBuf> RpcSerde::Preamble(bool secure) {
+  auto magic = IOBuf::copyBuffer(PREAMBLE, 0, 2);
+  magic->append(2);
+  RWPrivateCursor c(magic.get());
+  c.skip(4);
+  // Version
+  c.write(RPC_VERSION);
+  if (secure) {
+    // for now support only KERBEROS (DIGEST is not supported)
+    c.write(KERBEROS_AUTH_TYPE);
+  } else {
+    c.write(DEFAULT_AUTH_TYPE);
+  }
+  return magic;
+}
+
+std::unique_ptr<IOBuf> RpcSerde::Header(const std::string &user) {
+  pb::ConnectionHeader h;
+
+  // TODO(eclark): Make this not a total lie.
+  h.mutable_user_info()->set_effective_user(user);
+  // The service name that we want to talk to.
+  //
+  // Right now we're completely ignoring the service interface.
+  // That may or may not be the correct thing to do.
+  // It worked for a while with the java client; until it
+  // didn't.
+  // TODO: send the service name and user from the RpcClient
+  h.set_service_name(INTERFACE);
+
+  std::unique_ptr<pb::VersionInfo> version_info = CreateVersionInfo();
+
+  h.set_allocated_version_info(version_info.release());
+
+  if (codec_ != nullptr) {
+    h.set_cell_block_codec_class(codec_->java_class_name());
+  }
+  return PrependLength(SerializeMessage(h));
+}
+
+std::unique_ptr<pb::VersionInfo> RpcSerde::CreateVersionInfo() {
+  std::unique_ptr<pb::VersionInfo> version_info = std::make_unique<pb::VersionInfo>();
+  version_info->set_user(Version::user);
+  version_info->set_revision(Version::revision);
+  version_info->set_url(Version::url);
+  version_info->set_date(Version::date);
+  version_info->set_src_checksum(Version::src_checksum);
+  version_info->set_version(Version::version);
+
+  std::string version{Version::version};
+  std::vector<std::string> version_parts;
+  boost::split(version_parts, version, boost::is_any_of("."), boost::token_compress_on);
+  uint32_t major_version = 0, minor_version = 0;
+  if (version_parts.size() >= 2) {
+    version_info->set_version_major(folly::to<uint32_t>(version_parts[0]));
+    version_info->set_version_minor(folly::to<uint32_t>(version_parts[1]));
+  }
+
+  VLOG(1) << "Client VersionInfo:" << version_info->ShortDebugString();
+  return version_info;
+}
+
+std::unique_ptr<IOBuf> RpcSerde::Request(const uint32_t call_id, const std::string &method,
+                                         const Message *msg) {
+  pb::RequestHeader rq;
+  rq.set_method_name(method);
+  rq.set_call_id(call_id);
+  rq.set_request_param(msg != nullptr);
+  auto ser_header = SerializeDelimited(rq);
+  if (msg != nullptr) {
+    auto ser_req = SerializeDelimited(*msg);
+    ser_header->appendChain(std::move(ser_req));
+  }
+
+  return PrependLength(std::move(ser_header));
+}
+
+std::unique_ptr<folly::IOBuf> RpcSerde::Response(const uint32_t call_id,
+                                                 const google::protobuf::Message *msg) {
+  pb::ResponseHeader rh;
+  rh.set_call_id(call_id);
+  auto ser_header = SerializeDelimited(rh);
+  auto ser_resp = SerializeDelimited(*msg);
+  ser_header->appendChain(std::move(ser_resp));
+
+  return PrependLength(std::move(ser_header));
+}
+
+std::unique_ptr<folly::IOBuf> RpcSerde::Response(const uint32_t call_id,
+                                                 const google::protobuf::Message *msg,
+                                                 const folly::exception_wrapper &exception) {
+  /* create ResponseHeader */
+  pb::ResponseHeader rh;
+  rh.set_call_id(call_id);
+
+  /* create ExceptionResponse */
+  if (bool(exception)) {
+    VLOG(1) << "packing ExceptionResponse";
+    auto exception_response = new pb::ExceptionResponse();
+    exception_response->set_exception_class_name(exception.class_name().c_str());
+    exception_response->set_stack_trace(exception.what().c_str());
+    rh.set_allocated_exception(exception_response);
+  }
+
+  /* serialize Response header and body */
+  auto ser_header = SerializeDelimited(rh);
+  auto ser_resp = SerializeDelimited(*msg);
+  ser_header->appendChain(std::move(ser_resp));
+
+  VLOG(3) << "Converted hbase::Response to folly::IOBuf";
+  return PrependLength(std::move(ser_header));
+}
+
+std::unique_ptr<CellScanner> RpcSerde::CreateCellScanner(std::unique_ptr<folly::IOBuf> buf,
+                                                         uint32_t offset, uint32_t length) {
+  if (codec_ == nullptr) {
+    return nullptr;
+  }
+  return codec_->CreateDecoder(std::move(buf), offset, length);
+}
+
+std::unique_ptr<IOBuf> RpcSerde::PrependLength(std::unique_ptr<IOBuf> msg) {
+  // Java ints are 4 long. So create a buffer that large
+  auto len_buf = IOBuf::create(4);
+  // Then make those bytes visible.
+  len_buf->append(4);
+
+  RWPrivateCursor c(len_buf.get());
+  // Get the size of the data to be pushed out the network.
+  auto size = msg->computeChainDataLength();
+
+  // Write the length to this IOBuf.
+  c.writeBE(static_cast<uint32_t>(size));
+
+  // Then attach the origional to the back of len_buf
+  len_buf->appendChain(std::move(msg));
+  return len_buf;
+}
+
+std::unique_ptr<IOBuf> RpcSerde::SerializeDelimited(const Message &msg) {
+  // Get the buffer size needed for just the message.
+  int msg_size = msg.ByteSize();
+  int buf_size = CodedOutputStream::VarintSize32(msg_size) + msg_size;
+
+  // Create a buffer big enough to hold the varint and the object.
+  auto buf = IOBuf::create(buf_size);
+  buf->append(buf_size);
+
+  // Create the array output stream.
+  ArrayOutputStream aos{buf->writableData(), static_cast<int>(buf->length())};
+  // Wrap the ArrayOuputStream in the coded output stream to allow writing
+  // Varint32
+  CodedOutputStream cos{&aos};
+
+  // Write out the size.
+  cos.WriteVarint32(msg_size);
+
+  // Now write the rest out.
+  // We're using the protobuf output streams here to keep track
+  // of where in the output array we are rather than IOBuf.
+  msg.SerializeWithCachedSizesToArray(cos.GetDirectBufferForNBytesAndAdvance(msg_size));
+
+  // Return the buffer.
+  return buf;
+}
+// TODO(eclark): Make this 1 copy.
+std::unique_ptr<IOBuf> RpcSerde::SerializeMessage(const Message &msg) {
+  auto buf = IOBuf::copyBuffer(msg.SerializeAsString());
+  return buf;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/server-name-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/server-name-test.cc b/hbase-native-client/src/hbase/serde/server-name-test.cc
new file mode 100644
index 0000000..12d3287
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/server-name-test.cc
@@ -0,0 +1,47 @@
+/*
+ * 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 "hbase/serde/server-name.h"
+
+#include <gtest/gtest.h>
+#include <string>
+
+using hbase::pb::ServerName;
+
+TEST(TestServerName, TestMakeServerName) {
+  auto sn = folly::to<ServerName>("test:123");
+
+  ASSERT_EQ("test", sn.host_name());
+  ASSERT_EQ(123, sn.port());
+}
+
+TEST(TestServerName, TestIps) {
+  auto sn = folly::to<ServerName>("127.0.0.1:999");
+  ASSERT_EQ("127.0.0.1", sn.host_name());
+  ASSERT_EQ(999, sn.port());
+}
+
+TEST(TestServerName, TestThrow) { ASSERT_ANY_THROW(folly::to<ServerName>("Ther's no colon here")); }
+
+TEST(TestServerName, TestIPV6) {
+  auto sn = folly::to<ServerName>("[::::1]:123");
+
+  ASSERT_EQ("[::::1]", sn.host_name());
+  ASSERT_EQ(123, sn.port());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/table-name-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/table-name-test.cc b/hbase-native-client/src/hbase/serde/table-name-test.cc
new file mode 100644
index 0000000..77bd6c2
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/table-name-test.cc
@@ -0,0 +1,54 @@
+/*
+ * 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 <folly/Conv.h>
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include "hbase/serde/table-name.h"
+
+using namespace hbase;
+using hbase::pb::TableName;
+
+TEST(TestTableName, TestToStringNoDefault) {
+  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;
+  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);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/zk-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/zk-deserializer-test.cc b/hbase-native-client/src/hbase/serde/zk-deserializer-test.cc
new file mode 100644
index 0000000..141efce
--- /dev/null
+++ b/hbase-native-client/src/hbase/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 "hbase/serde/zk.h"
+
+#include <folly/Logging.h>
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <gtest/gtest.h>
+
+#include "hbase/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());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/serde/zk.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/serde/zk.cc b/hbase-native-client/src/hbase/serde/zk.cc
new file mode 100644
index 0000000..b962cc5
--- /dev/null
+++ b/hbase-native-client/src/hbase/serde/zk.cc
@@ -0,0 +1,77 @@
+/*
+ * 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 "hbase/serde/zk.h"
+
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <google/protobuf/message.h>
+
+#include <string>
+
+using std::runtime_error;
+
+namespace hbase {
+
+static const std::string MAGIC_STRING = "PBUF";
+
+bool ZkDeserializer::Parse(folly::IOBuf *buf, google::protobuf::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.
+
+  folly::io::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;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/test-util/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/test-util/BUCK b/hbase-native-client/src/hbase/test-util/BUCK
new file mode 100644
index 0000000..f1aedab
--- /dev/null
+++ b/hbase-native-client/src/hbase/test-util/BUCK
@@ -0,0 +1,53 @@
+##
+# 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.
+import os
+
+cxx_library(
+    name="test-util",
+    header_namespace="hbase/test-util",
+    exported_headers=["test-util.h", "mini-cluster.h"],
+    srcs=["test-util.cc", "mini-cluster.cc"],
+    deps=[
+        "//third-party:folly",
+        "//src/hbase/client:client",
+    ],
+    preprocessor_flags=[
+        '-I' + os.environ['JAVA_HOME'] + '/include',
+        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
+        '-I' + os.environ['JAVA_HOME'] + '/include/linux'
+    ],
+    exported_preprocessor_flags=[
+        '-I' + os.environ['JAVA_HOME'] + '/include',
+        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
+        '-I' + os.environ['JAVA_HOME'] + '/include/linux'
+    ],
+    compiler_flags=[
+        '-I' + os.environ['JAVA_HOME'] + '/include',
+        '-I' + os.environ['JAVA_HOME'] + '/include/darwin',
+        '-I' + os.environ['JAVA_HOME'] + '/include/linux', '-ggdb'
+    ],
+    linker_flags=[
+        '-ljvm', '-L' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server',
+        '-ggdb'
+    ],
+    exported_linker_flags=[
+        '-ljvm', '-L' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server',
+        '-Wl,-rpath=' + os.environ['JAVA_HOME'] + '/jre/lib/amd64/server'
+    ],
+    visibility=[
+        'PUBLIC',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/test-util/mini-cluster.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/test-util/mini-cluster.cc b/hbase-native-client/src/hbase/test-util/mini-cluster.cc
new file mode 100644
index 0000000..1e491a2
--- /dev/null
+++ b/hbase-native-client/src/hbase/test-util/mini-cluster.cc
@@ -0,0 +1,311 @@
+/*
+ * 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 "hbase/test-util/mini-cluster.h"
+#include <fcntl.h>
+#include <glog/logging.h>
+#include <boost/filesystem/fstream.hpp>
+#include <boost/filesystem/operations.hpp>
+#include <fstream>
+
+using hbase::MiniCluster;
+
+JNIEnv *MiniCluster::CreateVM(JavaVM **jvm) {
+  JavaVMInitArgs args;
+  JavaVMOption jvm_options;
+  args.version = JNI_VERSION_1_6;
+  args.nOptions = 1;
+  char *classpath = getenv("CLASSPATH");
+  std::string clspath;
+  if (classpath == NULL || strstr(classpath, "-tests.jar") == NULL) {
+    std::string clsPathFilePath("../target/cached_classpath.txt");
+    std::ifstream fd(clsPathFilePath);
+    std::string prefix("");
+    if (fd.is_open()) {
+      if (classpath == NULL) {
+        LOG(INFO) << "got empty classpath";
+      } else {
+        // prefix bootstrapper.jar
+        prefix.assign(classpath);
+      }
+      std::string line;
+      if (getline(fd, line)) {
+        clspath = prefix + ":" + line;
+        int ret = setenv("CLASSPATH", clspath.c_str(), 1);
+        LOG(INFO) << "set clspath " << ret;
+      } else {
+        LOG(INFO) << "nothing read from " << clsPathFilePath;
+        exit(-1);
+      }
+    } else {
+      LOG(INFO) << "nothing read from " << clsPathFilePath;
+      exit(-1);
+    }
+    fd.close();
+  }
+
+  auto options = std::string{"-Djava.class.path="} + clspath;
+  jvm_options.optionString = const_cast<char *>(options.c_str());
+  args.options = &jvm_options;
+  args.ignoreUnrecognized = 0;
+  int rv;
+  rv = JNI_CreateJavaVM(jvm, reinterpret_cast<void **>(&env_), &args);
+  CHECK(rv >= 0 && env_);
+  return env_;
+}
+
+MiniCluster::~MiniCluster() {
+  if (jvm_ != NULL) {
+    jvm_->DestroyJavaVM();
+    jvm_ = NULL;
+  }
+  env_ = nullptr;
+}
+
+void MiniCluster::Setup() {
+  jmethodID constructor;
+  pthread_mutex_lock(&count_mutex_);
+  if (env_ == NULL) {
+    env_ = CreateVM(&jvm_);
+    if (env_ == NULL) {
+      exit(-1);
+    }
+    testing_util_class_ = env_->FindClass("org/apache/hadoop/hbase/HBaseTestingUtility");
+    // this should be converted to a globalref I think to avoid the underlying java obj getting
+    // GC'ed
+    if (testing_util_class_ == NULL) {
+      LOG(INFO) << "Couldn't find class HBaseTestingUtility";
+      exit(-1);
+    }
+    jmethodID mid = env_->GetStaticMethodID(testing_util_class_, "createLocalHTU",
+                                            "()Lorg/apache/hadoop/hbase/HBaseTestingUtility;");
+    htu_ = env_->CallStaticObjectMethod(testing_util_class_, mid);
+    // this should be converted to a globalref I think to avoid the underlying java obj getting
+    // GC'ed
+    if (htu_ == NULL) {
+      LOG(INFO) << "Couldn't invoke method createLocalHTU in HBaseTestingUtility";
+      exit(-1);
+    }
+    get_conn_mid_ = env_->GetMethodID(testing_util_class_, "getConnection",
+                                      "()Lorg/apache/hadoop/hbase/client/Connection;");
+    jclass conn_class = env_->FindClass("org/apache/hadoop/hbase/client/Connection");
+    get_admin_mid_ =
+        env_->GetMethodID(conn_class, "getAdmin", "()Lorg/apache/hadoop/hbase/client/Admin;");
+    get_table_mid_ = env_->GetMethodID(
+        conn_class, "getTable",
+        "(Lorg/apache/hadoop/hbase/TableName;)Lorg/apache/hadoop/hbase/client/Table;");
+    if (get_table_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find getConnection";
+      exit(-1);
+    }
+    jclass adminClass = env_->FindClass("org/apache/hadoop/hbase/client/Admin");
+    move_mid_ = env_->GetMethodID(adminClass, "move", "([B[B)V");
+    if (move_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find move";
+      exit(-1);
+    }
+    create_table_mid_ =
+        env_->GetMethodID(testing_util_class_, "createTable",
+                          "(Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;)Lorg/"
+                          "apache/hadoop/hbase/client/Table;");
+    create_table_families_mid_ = env_->GetMethodID(testing_util_class_, "createTable",
+                                                   "(Lorg/apache/hadoop/hbase/TableName;[[B)Lorg/"
+                                                   "apache/hadoop/hbase/client/Table;");
+    create_table_with_split_mid_ = env_->GetMethodID(
+        testing_util_class_, "createTable",
+        "(Lorg/apache/hadoop/hbase/TableName;[[B[[B)Lorg/apache/hadoop/hbase/client/Table;");
+    if (create_table_with_split_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find method createTable with split";
+      exit(-1);
+    }
+
+    table_name_class_ = env_->FindClass("org/apache/hadoop/hbase/TableName");
+    tbl_name_value_of_mid_ = env_->GetStaticMethodID(
+        table_name_class_, "valueOf", "(Ljava/lang/String;)Lorg/apache/hadoop/hbase/TableName;");
+    if (tbl_name_value_of_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find method valueOf in TableName";
+      exit(-1);
+    }
+    jclass hbaseMiniClusterClass = env_->FindClass("org/apache/hadoop/hbase/MiniHBaseCluster");
+    stop_rs_mid_ =
+        env_->GetMethodID(hbaseMiniClusterClass, "stopRegionServer",
+                          "(I)Lorg/apache/hadoop/hbase/util/JVMClusterUtil$RegionServerThread;");
+    get_conf_mid_ = env_->GetMethodID(hbaseMiniClusterClass, "getConfiguration",
+                                      "()Lorg/apache/hadoop/conf/Configuration;");
+
+    conf_class_ = env_->FindClass("org/apache/hadoop/conf/Configuration");
+    set_conf_mid_ =
+        env_->GetMethodID(conf_class_, "set", "(Ljava/lang/String;Ljava/lang/String;)V");
+    if (set_conf_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find method getConf in MiniHBaseCluster";
+      exit(-1);
+    }
+    conf_get_mid_ = env_->GetMethodID(conf_class_, "get", "(Ljava/lang/String;)Ljava/lang/String;");
+
+    jclass tableClass = env_->FindClass("org/apache/hadoop/hbase/client/Table");
+    put_mid_ = env_->GetMethodID(tableClass, "put", "(Lorg/apache/hadoop/hbase/client/Put;)V");
+    jclass connFactoryClass = env_->FindClass("org/apache/hadoop/hbase/client/ConnectionFactory");
+    create_conn_mid_ = env_->GetStaticMethodID(connFactoryClass, "createConnection",
+                                               "()Lorg/apache/hadoop/hbase/client/Connection;");
+    if (create_conn_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find createConnection";
+      exit(-1);
+    }
+    put_class_ = env_->FindClass("org/apache/hadoop/hbase/client/Put");
+    put_ctor_ = env_->GetMethodID(put_class_, "<init>", "([B)V");
+    add_col_mid_ =
+        env_->GetMethodID(put_class_, "addColumn", "([B[B[B)Lorg/apache/hadoop/hbase/client/Put;");
+    if (add_col_mid_ == NULL) {
+      LOG(INFO) << "Couldn't find method addColumn";
+      exit(-1);
+    }
+  }
+  pthread_mutex_unlock(&count_mutex_);
+}
+
+jobject MiniCluster::htu() {
+  Setup();
+  return htu_;
+}
+
+JNIEnv *MiniCluster::env() {
+  Setup();
+  return env_;
+}
+// converts C char* to Java byte[]
+jbyteArray MiniCluster::StrToByteChar(const std::string &str) {
+  if (str.length() == 0) {
+    return nullptr;
+  }
+  int n = str.length();
+  jbyteArray arr = env_->NewByteArray(n);
+  env_->SetByteArrayRegion(arr, 0, n, reinterpret_cast<const jbyte *>(str.c_str()));
+  return arr;
+}
+
+jobject MiniCluster::CreateTable(const std::string &table, const std::string &family) {
+  jstring table_name_str = env_->NewStringUTF(table.c_str());
+  jobject table_name =
+      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
+  jstring family_str = env_->NewStringUTF(family.c_str());
+  jobject table_obj = env_->CallObjectMethod(htu_, create_table_mid_, table_name, family_str);
+  return table_obj;
+}
+
+jobject MiniCluster::CreateTable(const std::string &table,
+                                 const std::vector<std::string> &families) {
+  jstring table_name_str = env_->NewStringUTF(table.c_str());
+  jobject table_name =
+      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
+  jclass array_element_type = env_->FindClass("[B");
+  jobjectArray family_array = env_->NewObjectArray(families.size(), array_element_type, nullptr);
+  int i = 0;
+  for (auto family : families) {
+    env_->SetObjectArrayElement(family_array, i++, StrToByteChar(family));
+  }
+  jobject table_obj =
+      env_->CallObjectMethod(htu_, create_table_families_mid_, table_name, family_array);
+  return table_obj;
+}
+
+jobject MiniCluster::CreateTable(const std::string &table, const std::string &family,
+                                 const std::vector<std::string> &keys) {
+  std::vector<std::string> families{};
+  families.push_back(std::string{family});
+  return CreateTable(table, families, keys);
+}
+
+jobject MiniCluster::CreateTable(const std::string &table, const std::vector<std::string> &families,
+                                 const std::vector<std::string> &keys) {
+  jstring table_name_str = env_->NewStringUTF(table.c_str());
+  jobject table_name =
+      env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_, table_name_str);
+  jclass array_element_type = env_->FindClass("[B");
+
+  int i = 0;
+  jobjectArray family_array = env_->NewObjectArray(families.size(), array_element_type, nullptr);
+  for (auto family : families) {
+    env_->SetObjectArrayElement(family_array, i++, StrToByteChar(family));
+  }
+
+  jobjectArray key_array = env_->NewObjectArray(keys.size(), array_element_type, nullptr);
+
+  i = 0;
+  for (auto key : keys) {
+    env_->SetObjectArrayElement(key_array, i++, StrToByteChar(key));
+  }
+
+  jobject tbl = env_->CallObjectMethod(htu_, create_table_with_split_mid_, table_name, family_array,
+                                       key_array);
+  return tbl;
+}
+
+jobject MiniCluster::StopRegionServer(int idx) {
+  env();
+  return env_->CallObjectMethod(cluster_, stop_rs_mid_, (jint)idx);
+}
+
+// returns the Configuration for the cluster
+jobject MiniCluster::GetConf() {
+  env();
+  return env_->CallObjectMethod(cluster_, get_conf_mid_);
+}
+// return the Admin instance for the local cluster
+jobject MiniCluster::admin() {
+  env();
+  jobject conn = env_->CallObjectMethod(htu(), get_conn_mid_);
+  jobject admin = env_->CallObjectMethod(conn, get_admin_mid_);
+  return admin;
+}
+
+// moves region to server
+void MiniCluster::MoveRegion(const std::string &region, const std::string &server) {
+  jobject admin_ = admin();
+  env_->CallObjectMethod(admin_, move_mid_, StrToByteChar(region), StrToByteChar(server));
+}
+
+jobject MiniCluster::StartCluster(int num_region_servers) {
+  env();
+  jmethodID mid = env_->GetMethodID(testing_util_class_, "startMiniCluster",
+                                    "(I)Lorg/apache/hadoop/hbase/MiniHBaseCluster;");
+  if (mid == NULL) {
+    LOG(INFO) << "Couldn't find method startMiniCluster in the class HBaseTestingUtility";
+    exit(-1);
+  }
+  cluster_ = env_->CallObjectMethod(htu(), mid, static_cast<jint>(num_region_servers));
+  return cluster_;
+}
+
+void MiniCluster::StopCluster() {
+  env();
+  jmethodID mid = env_->GetMethodID(testing_util_class_, "shutdownMiniCluster", "()V");
+  env_->CallVoidMethod(htu(), mid);
+  if (jvm_ != NULL) {
+    jvm_->DestroyJavaVM();
+    jvm_ = NULL;
+  }
+}
+
+const std::string MiniCluster::GetConfValue(const std::string &key) {
+  jobject conf = GetConf();
+  jstring jval =
+      (jstring)env_->CallObjectMethod(conf, conf_get_mid_, env_->NewStringUTF(key.c_str()));
+  const char *val = env_->GetStringUTFChars(jval, 0);
+  return val;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/test-util/mini-cluster.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/test-util/mini-cluster.h b/hbase-native-client/src/hbase/test-util/mini-cluster.h
new file mode 100644
index 0000000..6b4547c
--- /dev/null
+++ b/hbase-native-client/src/hbase/test-util/mini-cluster.h
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <jni.h>
+#include <string>
+#include <vector>
+
+namespace hbase {
+
+class MiniCluster {
+ public:
+  virtual ~MiniCluster();
+  jobject StartCluster(int32_t num_region_servers);
+  void StopCluster();
+  jobject CreateTable(const std::string &table, const std::string &family);
+  jobject CreateTable(const std::string &table, const std::vector<std::string> &families);
+  jobject CreateTable(const std::string &table, const std::string &family,
+                      const std::vector<std::string> &keys);
+  jobject CreateTable(const std::string &table, const std::vector<std::string> &families,
+                      const std::vector<std::string> &keys);
+  jobject StopRegionServer(int idx);
+
+  // moves region to server
+  void MoveRegion(const std::string &region, const std::string &server);
+  // returns the Configuration instance for the cluster
+  jobject GetConf();
+  // returns the value for config key retrieved from cluster
+  const std::string GetConfValue(const std::string &key);
+
+ private:
+  JNIEnv *env_;
+  jclass testing_util_class_;
+  jclass table_name_class_;
+  jclass put_class_;
+  jclass conf_class_;
+  jmethodID stop_rs_mid_;
+  jmethodID get_conf_mid_;
+  jmethodID set_conf_mid_;
+  jmethodID tbl_name_value_of_mid_;
+  jmethodID create_table_mid_;
+  jmethodID create_table_families_mid_;
+  jmethodID create_table_with_split_mid_;
+  jmethodID put_mid_;
+  jmethodID put_ctor_;
+  jmethodID add_col_mid_;
+  jmethodID create_conn_mid_;
+  jmethodID get_conn_mid_;
+  jmethodID get_table_mid_;
+  jmethodID conf_get_mid_;
+  jmethodID get_admin_mid_;
+  jmethodID move_mid_;
+  jmethodID str_ctor_mid_;
+  jobject htu_;
+  jobject cluster_;
+  pthread_mutex_t count_mutex_;
+  JavaVM *jvm_;
+  JNIEnv *CreateVM(JavaVM **jvm);
+  void Setup();
+  jobject htu();
+  JNIEnv *env();
+  jbyteArray StrToByteChar(const std::string &str);
+  jobject admin();
+};
+} /*namespace hbase*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/test-util/test-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/test-util/test-util.cc b/hbase-native-client/src/hbase/test-util/test-util.cc
new file mode 100644
index 0000000..ebaf701
--- /dev/null
+++ b/hbase-native-client/src/hbase/test-util/test-util.cc
@@ -0,0 +1,105 @@
+/*
+ * 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 "hbase/test-util/test-util.h"
+#include <string.h>
+
+#include <folly/Format.h>
+
+#include "hbase/client/zk-util.h"
+
+using hbase::TestUtil;
+using folly::Random;
+
+std::string TestUtil::RandString(int len) {
+  // Create the whole string.
+  // Filling everything with z's
+  auto s = std::string(len, 'z');
+
+  // Now pick a bunch of random numbers
+  for (int i = 0; i < len; i++) {
+    // use Folly's random to get the numbers
+    // as I don't want to have to learn
+    // all the cpp rand invocation magic.
+    auto r = Random::rand32('a', 'z');
+    // Cast that to ascii.
+    s[i] = static_cast<char>(r);
+  }
+  return s;
+}
+
+TestUtil::TestUtil() : temp_dir_(TestUtil::RandString()) {}
+
+TestUtil::~TestUtil() {
+  if (mini_) {
+    StopMiniCluster();
+    mini_ = nullptr;
+  }
+}
+
+void TestUtil::StartMiniCluster(int32_t num_region_servers) {
+  mini_ = std::make_unique<MiniCluster>();
+  mini_->StartCluster(num_region_servers);
+
+  conf()->Set(ZKUtil::kHBaseZookeeperQuorum_, mini_->GetConfValue(ZKUtil::kHBaseZookeeperQuorum_));
+  conf()->Set(ZKUtil::kHBaseZookeeperClientPort_,
+              mini_->GetConfValue(ZKUtil::kHBaseZookeeperClientPort_));
+}
+
+void TestUtil::StopMiniCluster() { mini_->StopCluster(); }
+
+void TestUtil::CreateTable(const std::string &table, const std::string &family) {
+  mini_->CreateTable(table, family);
+}
+
+void TestUtil::CreateTable(const std::string &table, const std::vector<std::string> &families) {
+  mini_->CreateTable(table, families);
+}
+
+void TestUtil::CreateTable(const std::string &table, const std::string &family,
+                           const std::vector<std::string> &keys) {
+  mini_->CreateTable(table, family, keys);
+}
+
+void TestUtil::CreateTable(const std::string &table, const std::vector<std::string> &families,
+                           const std::vector<std::string> &keys) {
+  mini_->CreateTable(table, families, keys);
+}
+
+void TestUtil::MoveRegion(const std::string &region, const std::string &server) {
+  mini_->MoveRegion(region, server);
+}
+
+void TestUtil::StartStandAloneInstance() {
+  auto p = temp_dir_.path().string();
+  auto cmd = std::string{"bin/start-local-hbase.sh " + p};
+  auto res_code = std::system(cmd.c_str());
+  CHECK_EQ(res_code, 0);
+}
+
+void TestUtil::StopStandAloneInstance() {
+  auto res_code = std::system("bin/stop-local-hbase.sh");
+  CHECK_EQ(res_code, 0);
+}
+
+void TestUtil::RunShellCmd(const std::string &command) {
+  auto cmd_string = folly::sformat("echo \"{}\" | ../bin/hbase shell", command);
+  auto res_code = std::system(cmd_string.c_str());
+  CHECK_EQ(res_code, 0);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/test-util/test-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/test-util/test-util.h b/hbase-native-client/src/hbase/test-util/test-util.h
new file mode 100644
index 0000000..7c57c28
--- /dev/null
+++ b/hbase-native-client/src/hbase/test-util/test-util.h
@@ -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.
+ *
+ */
+#pragma once
+
+#include <folly/Random.h>
+#include <folly/experimental/TestUtil.h>
+
+#include <cstdlib>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/configuration.h"
+#include "hbase/test-util/mini-cluster.h"
+
+namespace hbase {
+/**
+ * @brief Class to deal with a local instance cluster for testing.
+ */
+class TestUtil {
+ public:
+  TestUtil();
+
+  /**
+   * Destroying a TestUtil will spin down a cluster and remove the test dir.
+   */
+  ~TestUtil();
+
+  /**
+   * Create a random string. This random string is all letters, as such it is
+   * very good for use as a directory name.
+   */
+  static std::string RandString(int len = 32);
+
+  /**
+   * Returns the configuration to talk to the local cluster
+   */
+  std::shared_ptr<Configuration> conf() const { return conf_; }
+
+  /**
+   * Starts mini hbase cluster with specified number of region servers
+   */
+  void StartMiniCluster(int32_t num_region_servers);
+
+  void StopMiniCluster();
+  void CreateTable(const std::string &table, const std::string &family);
+  void CreateTable(const std::string &table, const std::vector<std::string> &families);
+  void CreateTable(const std::string &table, const std::string &family,
+                   const std::vector<std::string> &keys);
+  void CreateTable(const std::string &table, const std::vector<std::string> &families,
+                   const std::vector<std::string> &keys);
+
+  void StartStandAloneInstance();
+  void StopStandAloneInstance();
+  void RunShellCmd(const std::string &);
+  void MoveRegion(const std::string &region, const std::string &server);
+
+ private:
+  std::unique_ptr<MiniCluster> mini_;
+  folly::test::TemporaryDirectory temp_dir_;
+  std::shared_ptr<Configuration> conf_ = std::make_shared<Configuration>();
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/BUCK b/hbase-native-client/src/hbase/utils/BUCK
new file mode 100644
index 0000000..ab55d8f
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/BUCK
@@ -0,0 +1,57 @@
+##
+# 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.
+
+cxx_library(
+    name="utils",
+    exported_headers=[
+    ],
+    srcs=["bytes-util.cc", "connection-util.cc", "user-util.cc"],
+    deps=[
+        '//include/hbase/utils:utils',
+        '//third-party:folly',
+    ],
+    tests=[":user-util-test"],
+    linker_flags=['-L/usr/local/lib', '-lkrb5'],
+    exported_linker_flags=['-L/usr/local/lib', '-lkrb5'],
+    visibility=[
+        'PUBLIC',
+    ],
+    compiler_flags=['-Weffc++'],)
+cxx_test(
+    name="bytes-util-test",
+    srcs=[
+        "bytes-util-test.cc",
+    ],
+    deps=[
+        ":utils",
+    ],)
+cxx_test(
+    name="concurrent-map-test",
+    srcs=[
+        "concurrent-map-test.cc",
+    ],
+    deps=[
+        ":utils",
+    ],)
+cxx_test(
+    name="user-util-test",
+    srcs=[
+        "user-util-test.cc",
+    ],
+    deps=[
+        ":utils",
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/utils/bytes-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/utils/bytes-util-test.cc b/hbase-native-client/src/hbase/utils/bytes-util-test.cc
new file mode 100644
index 0000000..e601d2d
--- /dev/null
+++ b/hbase-native-client/src/hbase/utils/bytes-util-test.cc
@@ -0,0 +1,69 @@
+/*
+ * 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 <folly/Logging.h>
+#include <gtest/gtest.h>
+#include <string>
+
+#include "hbase/utils/bytes-util.h"
+
+using hbase::BytesUtil;
+
+TEST(TestBytesUtil, TestToStringBinary) {
+  std::string empty{""};
+  EXPECT_EQ(empty, BytesUtil::ToStringBinary(empty));
+
+  std::string foo_bar{"foo bar"};
+  EXPECT_EQ(foo_bar, BytesUtil::ToStringBinary(foo_bar));
+
+  std::string foo_bar2{"foo bar_/!@#$%^&*(){}[]|1234567890"};
+  EXPECT_EQ(foo_bar2, BytesUtil::ToStringBinary(foo_bar2));
+
+  char zero = 0;
+  EXPECT_EQ("\\x00", BytesUtil::ToStringBinary(std::string{zero}));
+
+  char max = 255;
+  EXPECT_EQ("\\xFF", BytesUtil::ToStringBinary(std::string{max}));
+
+  EXPECT_EQ("\\x00\\xFF", BytesUtil::ToStringBinary(std::string{zero} + std::string{max}));
+
+  EXPECT_EQ("foo_\\x00\\xFF_bar",
+            BytesUtil::ToStringBinary("foo_" + std::string{zero} + std::string{max} + "_bar"));
+}
+
+TEST(TestBytesUtil, TestToStringToInt64) {
+  int64_t num = 761235;
+  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
+
+  num = -56125;
+  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
+
+  num = 0;
+  EXPECT_EQ(num, BytesUtil::ToInt64(BytesUtil::ToString(num)));
+}
+
+TEST(TestBytesUtil, TestCreateClosestRowAfter) {
+  std::string empty{""};
+  EXPECT_EQ(BytesUtil::CreateClosestRowAfter(empty), std::string{'\0'});
+
+  std::string foo{"foo"};
+  EXPECT_EQ(BytesUtil::CreateClosestRowAfter(foo), std::string{"foo"} + '\0');
+
+  EXPECT_EQ("f\\x00", BytesUtil::ToStringBinary(BytesUtil::CreateClosestRowAfter("f")));
+}


[16/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/time-range-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/time-range-test.cc b/hbase-native-client/core/time-range-test.cc
deleted file mode 100644
index 136ab54..0000000
--- a/hbase-native-client/core/time-range-test.cc
+++ /dev/null
@@ -1,48 +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/time-range.h"
-
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-
-using namespace hbase;
-
-TEST(TimeRange, DefaultObject) {
-  TimeRange *timerange_def = nullptr;
-  ASSERT_NO_THROW(timerange_def = new TimeRange());
-
-  EXPECT_EQ(0, timerange_def->MinTimeStamp());
-  EXPECT_EQ(std::numeric_limits<int64_t>::max(), timerange_def->MaxTimeStamp());
-  EXPECT_NE(1000, timerange_def->MinTimeStamp());
-  EXPECT_NE(2000, timerange_def->MaxTimeStamp());
-  delete timerange_def;
-  timerange_def = nullptr;
-}
-
-TEST(TimeRange, Exception) {
-  // Negative Min TS
-  ASSERT_THROW(TimeRange(-1000, 2000), std::runtime_error);
-
-  // Negative Max TS
-  ASSERT_THROW(TimeRange(1000, -2000), std::runtime_error);
-
-  // Min TS > Max TS
-  ASSERT_THROW(TimeRange(10000, 2000), std::runtime_error);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/time-range.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/time-range.cc b/hbase-native-client/core/time-range.cc
deleted file mode 100644
index e6b789b..0000000
--- a/hbase-native-client/core/time-range.cc
+++ /dev/null
@@ -1,73 +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/time-range.h"
-#include <limits>
-#include <stdexcept>
-#include <string>
-
-namespace hbase {
-
-TimeRange::TimeRange()
-    : min_timestamp_(0L), max_timestamp_(std::numeric_limits<int64_t>::max()), all_time_(true) {}
-
-TimeRange::TimeRange(const TimeRange &tr) {
-  this->all_time_ = tr.all_time_;
-  this->max_timestamp_ = tr.max_timestamp_;
-  this->min_timestamp_ = tr.min_timestamp_;
-}
-
-TimeRange &TimeRange::operator=(const TimeRange &tr) {
-  this->all_time_ = tr.all_time_;
-  this->max_timestamp_ = tr.max_timestamp_;
-  this->min_timestamp_ = tr.min_timestamp_;
-  return *this;
-}
-
-TimeRange::~TimeRange() {}
-
-TimeRange::TimeRange(int64_t min_timestamp) {
-  this->min_timestamp_ = min_timestamp;
-  this->max_timestamp_ = std::numeric_limits<int64_t>::max();
-  this->all_time_ = false;
-}
-
-TimeRange::TimeRange(int64_t min_timestamp, int64_t max_timestamp) {
-  if (min_timestamp < 0 || max_timestamp < 0) {
-    throw std::runtime_error("Timestamp cannot be negative. min_timestamp: " +
-                             std::to_string(min_timestamp) + ", max_timestamp:" +
-                             std::to_string(max_timestamp));
-  }
-  if (max_timestamp < min_timestamp) {
-    throw std::runtime_error("max_timestamp [" + std::to_string(max_timestamp) +
-                             "] should be greater than min_timestamp [" +
-                             std::to_string(min_timestamp) + "]");
-  }
-
-  this->min_timestamp_ = min_timestamp;
-  this->max_timestamp_ = max_timestamp;
-  this->all_time_ = false;
-}
-
-int64_t TimeRange::MinTimeStamp() const { return this->min_timestamp_; }
-
-int64_t TimeRange::MaxTimeStamp() const { return this->max_timestamp_; }
-
-bool TimeRange::IsAllTime() const { return this->all_time_; }
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/time-range.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/time-range.h b/hbase-native-client/core/time-range.h
deleted file mode 100644
index d645ecd..0000000
--- a/hbase-native-client/core/time-range.h
+++ /dev/null
@@ -1,58 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-
-namespace hbase {
-class TimeRange {
- public:
-  /**
-   * @brief  Default constructor. Represents interval [0,
-   * std::numeric_limits<int64_t>::max())
-   * (allTime)
-   */
-  TimeRange();
-  TimeRange(const TimeRange &tr);
-  TimeRange &operator=(const TimeRange &tr);
-  /**
-   * @brief Represents interval [minStamp, std::numeric_limits<int64_t>::max())
-   * @param minStamp the minimum timestamp value, inclusive
-   */
-  explicit TimeRange(int64_t min_timestamp);
-  /**
-   * @brief Represents interval [minStamp, maxStamp)
-   * @param minStamp the minimum timestamp, inclusive
-   * @param maxStamp the maximum timestamp, exclusive
-   * @throws std::runtime_error if min_timestamp < 0 or max_timestamp < 0 or
-   * max_timestamp < min_timestamp
-   */
-  TimeRange(int64_t min_timestamp, int64_t max_timestamp);
-  int64_t MinTimeStamp() const;
-  int64_t MaxTimeStamp() const;
-  bool IsAllTime() const;
-  ~TimeRange();
-
- private:
-  int64_t min_timestamp_;
-  int64_t max_timestamp_;
-  bool all_time_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/zk-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/zk-util-test.cc b/hbase-native-client/core/zk-util-test.cc
deleted file mode 100644
index b0cefdc..0000000
--- a/hbase-native-client/core/zk-util-test.cc
+++ /dev/null
@@ -1,50 +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 <gtest/gtest.h>
-
-#include "core/zk-util.h"
-
-using hbase::Configuration;
-using hbase::ZKUtil;
-
-TEST(ZKUtilTest, ParseZooKeeperQuorum) {
-  Configuration conf{};
-  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1");
-  conf.SetInt(ZKUtil::kHBaseZookeeperClientPort_, 100);
-
-  ASSERT_EQ("s1:100", ZKUtil::ParseZooKeeperQuorum(conf));
-
-  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1:42");
-
-  ASSERT_EQ("s1:42", ZKUtil::ParseZooKeeperQuorum(conf));
-
-  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1,s2,s3");
-  ASSERT_EQ("s1:100,s2:100,s3:100", ZKUtil::ParseZooKeeperQuorum(conf));
-
-  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1:42,s2:42,s3:42");
-  ASSERT_EQ("s1:42,s2:42,s3:42", ZKUtil::ParseZooKeeperQuorum(conf));
-}
-
-TEST(ZKUtilTest, MetaZNode) {
-  Configuration conf{};
-  ASSERT_EQ("/hbase/meta-region-server", ZKUtil::MetaZNode(conf));
-
-  conf.Set(ZKUtil::kHBaseZnodeParent_, "/hbase-secure");
-  ASSERT_EQ("/hbase-secure/meta-region-server", ZKUtil::MetaZNode(conf));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/zk-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/zk-util.cc b/hbase-native-client/core/zk-util.cc
deleted file mode 100644
index d29c8c3..0000000
--- a/hbase-native-client/core/zk-util.cc
+++ /dev/null
@@ -1,62 +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/zk-util.h"
-
-#include <folly/Conv.h>
-#include <boost/algorithm/string.hpp>
-
-#include <vector>
-
-namespace hbase {
-
-/**
- * Returns a "proper" zookeeper quorum string, from hbase's broken quorum string formats. In
- * hbase.zookeeper.quorum, the ports are not listed explicitly per server (eg. s1,s2,s3),
- * however ZooKeeper expects the string of the format s1:2181,s2:2181,s3:2181. This code
- * appends the "clientPort" to each node in the quorum string if not there.
- */
-std::string ZKUtil::ParseZooKeeperQuorum(const hbase::Configuration& conf) {
-  auto zk_quorum = conf.Get(kHBaseZookeeperQuorum_, kDefHBaseZookeeperQuorum_);
-  auto zk_port = conf.GetInt(kHBaseZookeeperClientPort_, kDefHBaseZookeeperClientPort_);
-
-  std::vector<std::string> zk_quorum_parts;
-  boost::split(zk_quorum_parts, zk_quorum, boost::is_any_of(","), boost::token_compress_on);
-  std::vector<std::string> servers;
-  for (auto server : zk_quorum_parts) {
-    if (boost::contains(server, ":")) {
-      servers.push_back(server);
-    } else {
-      servers.push_back(server + ":" + folly::to<std::string>(zk_port));
-    }
-  }
-  return boost::join(servers, ",");
-}
-
-std::string ZKUtil::MetaZNode(const hbase::Configuration& conf) {
-  std::string zk_node = conf.Get(kHBaseZnodeParent_, kDefHBaseZnodeParent_) + "/";
-  zk_node += kHBaseMetaRegionServer_;
-  return zk_node;
-}
-
-int32_t ZKUtil::SessionTimeout(const hbase::Configuration& conf) {
-  return conf.GetInt(kHBaseZookeeperSessionTimeout_, kDefHBaseZookeeperSessionTimeout_);
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/zk-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/zk-util.h b/hbase-native-client/core/zk-util.h
deleted file mode 100644
index 403fbe4..0000000
--- a/hbase-native-client/core/zk-util.h
+++ /dev/null
@@ -1,46 +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.
- *
- */
-#pragma once
-
-#include <cstdlib>
-#include <string>
-#include "core/configuration.h"
-
-namespace hbase {
-
-class ZKUtil {
- public:
-  static constexpr const char* kHBaseZookeeperQuorum_ = "hbase.zookeeper.quorum";
-  static constexpr const char* kDefHBaseZookeeperQuorum_ = "localhost:2181";
-  static constexpr const char* kHBaseZookeeperClientPort_ = "hbase.zookeeper.property.clientPort";
-  static constexpr const int32_t kDefHBaseZookeeperClientPort_ = 2181;
-  static constexpr const char* kHBaseZnodeParent_ = "zookeeper.znode.parent";
-  static constexpr const char* kDefHBaseZnodeParent_ = "/hbase";
-  static constexpr const char* kHBaseMetaRegionServer_ = "meta-region-server";
-
-  static constexpr const char* kHBaseZookeeperSessionTimeout_ = "zookeeper.session.timeout";
-  static constexpr const int32_t kDefHBaseZookeeperSessionTimeout_ = 90000;
-
-  static std::string ParseZooKeeperQuorum(const hbase::Configuration& conf);
-
-  static std::string MetaZNode(const hbase::Configuration& conf);
-
-  static int32_t SessionTimeout(const hbase::Configuration& conf);
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/exceptions/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/exceptions/BUCK b/hbase-native-client/exceptions/BUCK
deleted file mode 100644
index 3d66d72..0000000
--- a/hbase-native-client/exceptions/BUCK
+++ /dev/null
@@ -1,39 +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.
-
-cxx_library(
-    name="exceptions",
-    exported_headers=[
-        "exception.h",
-    ],
-    srcs=[
-        "exception.cc",
-    ],
-    deps=[
-        "//third-party:folly",
-    ],
-    compiler_flags=['-Weffc++'],
-    visibility=['//core/...', '//connection/...'],)
-cxx_test(
-    name="exception-test",
-    srcs=[
-        "exception-test.cc",
-    ],
-    deps=[
-        ":exceptions",
-    ],
-    run_test_separately=True,)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/exceptions/exception-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/exceptions/exception-test.cc b/hbase-native-client/exceptions/exception-test.cc
deleted file mode 100644
index 583240e..0000000
--- a/hbase-native-client/exceptions/exception-test.cc
+++ /dev/null
@@ -1,64 +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 <gtest/gtest.h>
-
-#include "exceptions/exception.h"
-
-#include "folly/ExceptionWrapper.h"
-
-using hbase::ExceptionUtil;
-using hbase::IOException;
-using hbase::RemoteException;
-
-TEST(ExceptionUtilTest, IOExceptionShouldRetry) {
-  IOException ex{};
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_do_not_retry(true);
-  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_do_not_retry(false);
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  IOException ex2{"description", true};
-  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex2));
-
-  IOException ex3{"description", std::runtime_error("ex"), true};
-  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex3));
-}
-
-TEST(ExceptionUtilTest, RemoteExceptionShouldRetry) {
-  RemoteException ex{};
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_do_not_retry(true);
-  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_do_not_retry(false);
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_exception_class_name("org.apache.hadoop.hbase.FooException");
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_exception_class_name("org.apache.hadoop.hbase.NotServingRegionException");
-  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
-
-  ex.set_exception_class_name("org.apache.hadoop.hbase.UnknownRegionException");
-  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/exceptions/exception.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/exceptions/exception.cc b/hbase-native-client/exceptions/exception.cc
deleted file mode 100644
index c25acb4..0000000
--- a/hbase-native-client/exceptions/exception.cc
+++ /dev/null
@@ -1,128 +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 "exceptions/exception.h"
-
-namespace hbase {
-const std::vector<const char*> ExceptionUtil::kAllDoNotRetryIOExceptions = {
-    kDoNotRetryIOException,
-    kTableNotFoundException,
-    kTableNotEnabledException,
-    kCoprocessorException,
-    kBypassCoprocessorException,
-    kInvalidFamilyOperationException,
-    kServerTooBusyException,
-    kFailedSanityCheckException,
-    kCorruptHFileException,
-    kLabelAlreadyExistsException,
-    kFatalConnectionException,
-    kUnsupportedCryptoException,
-    kUnsupportedCellCodecException,
-    kEmptyServiceNameException,
-    kUnknownServiceException,
-    kWrongVersionException,
-    kBadAuthException,
-    kUnsupportedCompressionCodecException,
-    kDoNotRetryRegionException,
-    kRowTooBigException,
-    kRowTooBigExceptionDeprecated,
-    kUnknownRegionException,
-    kMergeRegionException,
-    kNoServerForRegionException,
-    kQuotaExceededException,
-    kSpaceLimitingException,
-    kThrottlingException,
-    kAccessDeniedException,
-    kUnknownProtocolException,
-    kRequestTooBigException,
-    kNotAllMetaRegionsOnlineException,
-    kConstraintException,
-    kNoSuchColumnFamilyException,
-    kLeaseException,
-    kInvalidLabelException,
-    kUnknownScannerException,
-    kScannerResetException,
-    kOutOfOrderScannerNextException};
-
-bool ExceptionUtil::ShouldRetry(const folly::exception_wrapper& error) {
-  bool do_not_retry = false;
-  error.with_exception(
-      [&](const IOException& ioe) { do_not_retry = do_not_retry || ioe.do_not_retry(); });
-  error.with_exception([&](const RemoteException& remote_ex) {
-    do_not_retry = do_not_retry || IsJavaDoNotRetryException(remote_ex.exception_class_name());
-  });
-  return !do_not_retry;
-}
-
-/**
- * Returns whether the java exception class extends DoNotRetryException.
- * In the java side, we just have a hierarchy of Exception classes that we use
- * both client side and server side. On the client side, we rethrow the server
- * side exception by un-wrapping the exception from a RemoteException or a ServiceException
- * (see ConnectionUtils.translateException() in Java).
- * Since this object-hierarchy info is not available in C++ side, we are doing a
- * very fragile catch-all list of all exception types in Java that extend the
- * DoNotRetryException class type.
- */
-bool ExceptionUtil::IsJavaDoNotRetryException(const std::string& java_class_name) {
-  for (auto exception : kAllDoNotRetryIOExceptions) {
-    if (java_class_name == exception) {
-      return true;
-    }
-  }
-  return false;
-}
-
-/**
- * Returns whether the scanner is closed when the client received the
- * remote exception.
- * Since the object-hierarchy info is not available in C++ side, we are doing a
- * very fragile catch-all list of all exception types in Java that extend these
- * three base classes: UnknownScannerException, NotServingRegionException,
- * RegionServerStoppedException
- */
-bool ExceptionUtil::IsScannerClosed(const folly::exception_wrapper& exception) {
-  bool scanner_closed = false;
-  exception.with_exception([&](const RemoteException& remote_ex) {
-    auto java_class = remote_ex.exception_class_name();
-    if (java_class == kUnknownScannerException || java_class == kNotServingRegionException ||
-        java_class == kRegionInRecoveryException || java_class == kRegionOpeningException ||
-        java_class == kRegionMovedException || java_class == kRegionServerStoppedException ||
-        java_class == kRegionServerAbortedException) {
-      scanner_closed = true;
-    }
-  });
-  return scanner_closed;
-}
-
-/**
- * Returns whether the wrapped exception is a java exception of type OutOfOrderScannerNextException
- * or ScannerResetException. These two exception types are thrown from the server side when the
- * scanner on the server side is closed.
- */
-bool ExceptionUtil::IsScannerOutOfOrder(const folly::exception_wrapper& exception) {
-  bool scanner_out_of_order = false;
-  exception.with_exception([&](const RemoteException& remote_ex) {
-    auto java_class = remote_ex.exception_class_name();
-    if (java_class == kOutOfOrderScannerNextException || java_class == kScannerResetException) {
-      scanner_out_of_order = true;
-    }
-  });
-  return scanner_out_of_order;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/exceptions/exception.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/exceptions/exception.h b/hbase-native-client/exceptions/exception.h
deleted file mode 100644
index bc3b291..0000000
--- a/hbase-native-client/exceptions/exception.h
+++ /dev/null
@@ -1,328 +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.
- *
- */
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/io/IOBuf.h>
-#include <exception>
-#include <memory>
-#include <string>
-#include <vector>
-
-namespace hbase {
-
-class ThrowableWithExtraContext {
- public:
-  ThrowableWithExtraContext(folly::exception_wrapper cause, const int64_t& when)
-      : cause_(cause), when_(when), extras_("") {}
-
-  ThrowableWithExtraContext(folly::exception_wrapper cause, const int64_t& when,
-                            const std::string& extras)
-      : cause_(cause), when_(when), extras_(extras) {}
-
-  virtual std::string ToString() {
-    // TODO:
-    // return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
-    return extras_ + ", " + cause_.what().toStdString();
-  }
-
-  virtual folly::exception_wrapper cause() { return cause_; }
-
- private:
-  folly::exception_wrapper cause_;
-  int64_t when_;
-  std::string extras_;
-};
-
-class IOException : public std::logic_error {
- public:
-  IOException() : logic_error(""), do_not_retry_(false) {}
-
-  explicit IOException(const std::string& what) : logic_error(what), do_not_retry_(false) {}
-
-  IOException(const std::string& what, bool do_not_retry)
-      : logic_error(what), do_not_retry_(do_not_retry) {}
-
-  IOException(const std::string& what, const folly::exception_wrapper& cause)
-      : logic_error(what), cause_(cause), do_not_retry_(false) {}
-
-  IOException(const std::string& what, folly::exception_wrapper cause, bool do_not_retry)
-      : logic_error(what), cause_(cause), do_not_retry_(do_not_retry) {}
-
-  virtual ~IOException() = default;
-
-  virtual folly::exception_wrapper cause() const { return cause_; }
-
-  bool do_not_retry() const { return do_not_retry_; }
-
-  IOException* set_do_not_retry(bool value) {
-    do_not_retry_ = value;
-    return this;
-  }
-
- private:
-  folly::exception_wrapper cause_;
-  // In case the exception is a RemoteException, do_not_retry information can come from
-  // the PB field in the RPC response, or it can be deduced from the Java-exception
-  // hierarchy in ExceptionUtil::ShouldRetry(). In case this is a client-side exception
-  // raised from the C++ internals, set this field so that the retrying callers can
-  // re-throw the exception without retrying.
-  bool do_not_retry_;
-};
-
-class RetriesExhaustedException : public IOException {
- public:
-  RetriesExhaustedException(const int& num_retries,
-                            std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions)
-      : IOException(GetMessage(num_retries, exceptions),
-                    exceptions->empty() ? folly::exception_wrapper{}
-                                        : (*exceptions)[exceptions->size() - 1].cause()),
-        num_retries_(num_retries) {}
-  virtual ~RetriesExhaustedException() = default;
-
-  int32_t num_retries() const { return num_retries_; }
-
- private:
-  std::string GetMessage(const int& num_retries,
-                         std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions) {
-    std::string buffer("Failed after attempts=");
-    buffer.append(std::to_string(num_retries + 1));
-    buffer.append(", exceptions:\n");
-    for (auto it = exceptions->begin(); it != exceptions->end(); it++) {
-      buffer.append(it->ToString());
-      buffer.append("\n");
-    }
-    return buffer;
-  }
-
- private:
-  int32_t num_retries_;
-};
-
-class ConnectionException : public IOException {
- public:
-  ConnectionException() {}
-
-  ConnectionException(const std::string& what) : IOException(what) {}
-
-  ConnectionException(const folly::exception_wrapper& cause) : IOException("", cause) {}
-
-  ConnectionException(const std::string& what, const folly::exception_wrapper& cause)
-      : IOException(what, cause) {}
-};
-
-class RemoteException : public IOException {
- public:
-  RemoteException() : IOException(), port_(0) {}
-
-  explicit RemoteException(const std::string& what) : IOException(what), port_(0) {}
-
-  RemoteException(const std::string& what, folly::exception_wrapper cause)
-      : IOException(what, cause), port_(0) {}
-
-  virtual ~RemoteException() = default;
-
-  std::string exception_class_name() const { return exception_class_name_; }
-
-  RemoteException* set_exception_class_name(const std::string& value) {
-    exception_class_name_ = value;
-    return this;
-  }
-
-  std::string stack_trace() const { return stack_trace_; }
-
-  RemoteException* set_stack_trace(const std::string& value) {
-    stack_trace_ = value;
-    return this;
-  }
-
-  std::string hostname() const { return hostname_; }
-
-  RemoteException* set_hostname(const std::string& value) {
-    hostname_ = value;
-    return this;
-  }
-
-  int port() const { return port_; }
-
-  RemoteException* set_port(int value) {
-    port_ = value;
-    return this;
-  }
-
- private:
-  std::string exception_class_name_;
-  std::string stack_trace_;
-  std::string hostname_;
-  int port_;
-};
-
-/**
- * Raised from the client side if we cannot find the table (does not have anything to
- * do with the Java exception of the same name).
- */
-class TableNotFoundException : public IOException {
- public:
-  explicit TableNotFoundException(const std::string& table_name)
-      : IOException("Table cannot be found:" + table_name, true) {}
-
-  virtual ~TableNotFoundException() = default;
-};
-
-/**
- * List of known exceptions from Java side, and Java-specific exception logic
- */
-class ExceptionUtil {
- private:
-  // unknown scanner and sub-classes
-  static constexpr const char* kUnknownScannerException =
-      "org.apache.hadoop.hbase.UnknownScannerException";
-
-  // not serving region and sub-classes
-  static constexpr const char* kNotServingRegionException =
-      "org.apache.hadoop.hbase.NotServingRegionException";
-  static constexpr const char* kRegionInRecoveryException =
-      "org.apache.hadoop.hbase.exceptions.RegionInRecoveryException";
-  static constexpr const char* kRegionOpeningException =
-      "org.apache.hadoop.hbase.exceptions.RegionOpeningException";
-  static constexpr const char* kRegionMovedException =
-      "org.apache.hadoop.hbase.exceptions.RegionMovedException";
-
-  // Region server stopped and sub-classes
-  static constexpr const char* kRegionServerStoppedException =
-      "org.apache.hadoop.hbase.regionserver.RegionServerStoppedException";
-  static constexpr const char* kRegionServerAbortedException =
-      "org.apache.hadoop.hbase.regionserver.RegionServerAbortedException";
-
-  // other scanner related exceptions
-  static constexpr const char* kOutOfOrderScannerNextException =
-      "org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException";
-  static constexpr const char* kScannerResetException =
-      "org.apache.hadoop.hbase.exceptions.ScannerResetException";
-
-  // All other DoNotRetryIOExceptions
-  static constexpr const char* kDoNotRetryIOException =
-      "org.apache.hadoop.hbase.DoNotRetryIOException";
-  static constexpr const char* kTableNotFoundException =
-      "org.apache.hadoop.hbase.TableNotFoundException";
-  static constexpr const char* kTableNotEnabledException =
-      "org.apache.hadoop.hbase.TableNotEnabledException";
-  static constexpr const char* kCoprocessorException =
-      "org.apache.hadoop.hbase.coprocessor.CoprocessorException";
-  static constexpr const char* kBypassCoprocessorException =
-      "org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException";
-  static constexpr const char* kInvalidFamilyOperationException =
-      "org.apache.hadoop.hbase.InvalidFamilyOperationException";
-  static constexpr const char* kServerTooBusyException =
-      "org.apache.hadoop.hbase.ipc.ServerTooBusyException";  // This should NOT be DNRIOE?
-  static constexpr const char* kFailedSanityCheckException =
-      "org.apache.hadoop.hbase.exceptions.FailedSanityCheckException";
-  static constexpr const char* kCorruptHFileException =
-      "org.apache.hadoop.hbase.io.hfile.CorruptHFileException";
-  static constexpr const char* kLabelAlreadyExistsException =
-      "org.apache.hadoop.hbase.security.visibility.LabelAlreadyExistsException";
-  static constexpr const char* kFatalConnectionException =
-      "org.apache.hadoop.hbase.ipc.FatalConnectionException";
-  static constexpr const char* kUnsupportedCryptoException =
-      "org.apache.hadoop.hbase.ipc.UnsupportedCryptoException";
-  static constexpr const char* kUnsupportedCellCodecException =
-      "org.apache.hadoop.hbase.ipc.UnsupportedCellCodecException";
-  static constexpr const char* kEmptyServiceNameException =
-      "org.apache.hadoop.hbase.ipc.EmptyServiceNameException";
-  static constexpr const char* kUnknownServiceException =
-      "org.apache.hadoop.hbase.ipc.UnknownServiceException";
-  static constexpr const char* kWrongVersionException =
-      "org.apache.hadoop.hbase.ipc.WrongVersionException";
-  static constexpr const char* kBadAuthException = "org.apache.hadoop.hbase.ipc.BadAuthException";
-  static constexpr const char* kUnsupportedCompressionCodecException =
-      "org.apache.hadoop.hbase.ipc.UnsupportedCompressionCodecException";
-  static constexpr const char* kDoNotRetryRegionException =
-      "org.apache.hadoop.hbase.client.DoNotRetryRegionException";
-  static constexpr const char* kRowTooBigException =
-      "org.apache.hadoop.hbase.client.RowTooBigException";
-  static constexpr const char* kRowTooBigExceptionDeprecated =
-      "org.apache.hadoop.hbase.regionserver.RowTooBigException";
-  static constexpr const char* kUnknownRegionException =
-      "org.apache.hadoop.hbase.UnknownRegionException";
-  static constexpr const char* kMergeRegionException =
-      "org.apache.hadoop.hbase.exceptions.MergeRegionException";
-  static constexpr const char* kNoServerForRegionException =
-      "org.apache.hadoop.hbase.client.NoServerForRegionException";
-  static constexpr const char* kQuotaExceededException =
-      "org.apache.hadoop.hbase.quotas.QuotaExceededException";
-  static constexpr const char* kSpaceLimitingException =
-      "org.apache.hadoop.hbase.quotas.SpaceLimitingException";
-  static constexpr const char* kThrottlingException =
-      "org.apache.hadoop.hbase.quotas.ThrottlingException";
-  static constexpr const char* kAccessDeniedException =
-      "org.apache.hadoop.hbase.security.AccessDeniedException";
-  static constexpr const char* kUnknownProtocolException =
-      "org.apache.hadoop.hbase.exceptions.UnknownProtocolException";
-  static constexpr const char* kRequestTooBigException =
-      "org.apache.hadoop.hbase.exceptions.RequestTooBigException";
-  static constexpr const char* kNotAllMetaRegionsOnlineException =
-      "org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException";
-  static constexpr const char* kConstraintException =
-      "org.apache.hadoop.hbase.constraint.ConstraintException";
-  static constexpr const char* kNoSuchColumnFamilyException =
-      "org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException";
-  static constexpr const char* kLeaseException =
-      "org.apache.hadoop.hbase.regionserver.LeaseException";
-  static constexpr const char* kInvalidLabelException =
-      "org.apache.hadoop.hbase.security.visibility.InvalidLabelException";
-
-  // TODO:
-  // These exceptions are not thrown in the regular read / write paths, although they are
-  // DoNotRetryIOExceptions. Add these to the list below in case we start doing Admin/DDL ops
-  // ReplicationPeerNotFoundException, XXXSnapshotException, NamespaceExistException,
-  // NamespaceNotFoundException, TableExistsException, TableNotDisabledException,
-  static const std::vector<const char*> kAllDoNotRetryIOExceptions;
-
- public:
-  /**
-   * Returns whether or not the exception should be retried by looking at the
-   * client-side IOException, or RemoteException coming from server side.
-   */
-  static bool ShouldRetry(const folly::exception_wrapper& error);
-
-  /**
-   * Returns whether the java exception class extends DoNotRetryException.
-   * In the java side, we just have a hierarchy of Exception classes that we use
-   * both client side and server side. On the client side, we rethrow the server
-   * side exception by un-wrapping the exception from a RemoteException or a ServiceException
-   * (see ConnectionUtils.translateException() in Java).
-   * Since this object-hierarchy info is not available in C++ side, we are doing a
-   * very fragile catch-all list of all exception types in Java that extend the
-   * DoNotRetryException class type.
-   */
-  static bool IsJavaDoNotRetryException(const std::string& java_class_name);
-
-  /**
-   * Returns whether the scanner is closed when the client received the
-   * remote exception.
-   * Since the object-hierarchy info is not available in C++ side, we are doing a
-   * very fragile catch-all list of all exception types in Java that extend these
-   * three base classes: UnknownScannerException, NotServingRegionException,
-   * RegionServerStoppedException
-   */
-  static bool IsScannerClosed(const folly::exception_wrapper& exception);
-
-  static bool IsScannerOutOfOrder(const folly::exception_wrapper& exception);
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/AccessControl.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/AccessControl.proto b/hbase-native-client/if/AccessControl.proto
deleted file mode 100644
index e67540b..0000000
--- a/hbase-native-client/if/AccessControl.proto
+++ /dev/null
@@ -1,123 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AccessControlProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message Permission {
-    enum Action {
-        READ = 0;
-        WRITE = 1;
-        EXEC = 2;
-        CREATE = 3;
-        ADMIN = 4;
-    }
-    enum Type {
-        Global = 1;
-        Namespace = 2;
-        Table = 3;
-    }
-    required Type type = 1;
-    optional GlobalPermission global_permission = 2;
-    optional NamespacePermission namespace_permission = 3;
-    optional TablePermission table_permission = 4;
-}
-
-message TablePermission {
-    optional TableName table_name = 1;
-    optional bytes family = 2;
-    optional bytes qualifier = 3;
-    repeated Permission.Action action = 4;
-}
-
-message NamespacePermission {
-    optional bytes namespace_name = 1;
-    repeated Permission.Action action = 2;
-}
-
-message GlobalPermission {
-    repeated Permission.Action action = 1;
-}
-
-message UserPermission {
-    required bytes user = 1;
-    required Permission permission = 3;
-}
-
-/**
- * Content of the /hbase/acl/<table or namespace> znode.
- */
-message UsersAndPermissions {
-  message UserPermissions {
-    required bytes user = 1;
-    repeated Permission permissions = 2;
-  }
-
-  repeated UserPermissions user_permissions = 1;
-}
-
-message GrantRequest {
-  required UserPermission user_permission = 1;
-}
-
-message GrantResponse {
-}
-
-message RevokeRequest {
-  required UserPermission user_permission = 1;
-}
-
-message RevokeResponse {
-}
-
-message GetUserPermissionsRequest {
-  optional Permission.Type type = 1;
-  optional TableName table_name = 2;
-  optional bytes namespace_name = 3;
-}
-
-message GetUserPermissionsResponse {
-  repeated UserPermission user_permission = 1;
-}
-
-message CheckPermissionsRequest {
-  repeated Permission permission = 1;
-}
-
-message CheckPermissionsResponse {
-}
-
-service AccessControlService {
-    rpc Grant(GrantRequest)
-      returns (GrantResponse);
-
-    rpc Revoke(RevokeRequest)
-      returns (RevokeResponse);
-
-    rpc GetUserPermissions(GetUserPermissionsRequest)
-      returns (GetUserPermissionsResponse);
-
-    rpc CheckPermissions(CheckPermissionsRequest)
-      returns (CheckPermissionsResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Admin.proto b/hbase-native-client/if/Admin.proto
deleted file mode 100644
index a1905a4..0000000
--- a/hbase-native-client/if/Admin.proto
+++ /dev/null
@@ -1,310 +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.
- */
-
-// This file contains protocol buffers that are used for Admin service.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AdminProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "Client.proto";
-import "HBase.proto";
-import "WAL.proto";
-
-message GetRegionInfoRequest {
-  required RegionSpecifier region = 1;
-  optional bool compaction_state = 2;
-}
-
-message GetRegionInfoResponse {
-  required RegionInfo region_info = 1;
-  optional CompactionState compaction_state = 2;
-  optional bool isRecovering = 3;
-
-  enum CompactionState {
-    NONE = 0;
-    MINOR = 1;
-    MAJOR = 2;
-    MAJOR_AND_MINOR = 3;
-  }
-}
-
-/**
- * Get a list of store files for a set of column families in a particular region.
- * If no column family is specified, get the store files for all column families.
- */
-message GetStoreFileRequest {
-  required RegionSpecifier region = 1;
-  repeated bytes family = 2;
-}
-
-message GetStoreFileResponse {
-  repeated string store_file = 1;
-}
-
-message GetOnlineRegionRequest {
-}
-
-message GetOnlineRegionResponse {
-  repeated RegionInfo region_info = 1;
-}
-
-message OpenRegionRequest {
-  repeated RegionOpenInfo open_info = 1;
-  // the intended server for this RPC.
-  optional uint64 serverStartCode = 2;
-  // wall clock time from master
-  optional uint64 master_system_time = 5;
-
-  message RegionOpenInfo {
-    required RegionInfo region = 1;
-    optional uint32 version_of_offline_node = 2;
-    repeated ServerName favored_nodes = 3;
-    // open region for distributedLogReplay
-    optional bool openForDistributedLogReplay = 4;
-  }
-}
-
-message OpenRegionResponse {
-  repeated RegionOpeningState opening_state = 1;
-
-  enum RegionOpeningState {
-    OPENED = 0;
-    ALREADY_OPENED = 1;
-    FAILED_OPENING = 2;
-  }
-}
-
-message WarmupRegionRequest {
-
-    required RegionInfo regionInfo = 1;
-}
-
-message WarmupRegionResponse {
-}
-
-/**
- * Closes the specified region and will use or not use ZK during the close
- * according to the specified flag.
- */
-message CloseRegionRequest {
-  required RegionSpecifier region = 1;
-  optional uint32 version_of_closing_node = 2;
-  optional bool transition_in_ZK = 3 [default = true];
-  optional ServerName destination_server = 4;
-  // the intended server for this RPC.
-  optional uint64 serverStartCode = 5;
-}
-
-message CloseRegionResponse {
-  required bool closed = 1;
-}
-
-/**
- * Flushes the MemStore of the specified region.
- * <p>
- * This method is synchronous.
- */
-message FlushRegionRequest {
-  required RegionSpecifier region = 1;
-  optional uint64 if_older_than_ts = 2;
-  optional bool write_flush_wal_marker = 3; // whether to write a marker to WAL even if not flushed
-}
-
-message FlushRegionResponse {
-  required uint64 last_flush_time = 1;
-  optional bool flushed = 2;
-  optional bool wrote_flush_wal_marker = 3;
-}
-
-/**
- * Splits the specified region.
- * <p>
- * This method currently flushes the region and then forces a compaction which
- * will then trigger a split.  The flush is done synchronously but the
- * compaction is asynchronous.
- */
-message SplitRegionRequest {
-  required RegionSpecifier region = 1;
-  optional bytes split_point = 2;
-}
-
-message SplitRegionResponse {
-}
-
-/**
- * Compacts the specified region.  Performs a major compaction if specified.
- * <p>
- * This method is asynchronous.
- */
-message CompactRegionRequest {
-  required RegionSpecifier region = 1;
-  optional bool major = 2;
-  optional bytes family = 3;
-}
-
-message CompactRegionResponse {
-}
-
-message UpdateFavoredNodesRequest {
-  repeated RegionUpdateInfo update_info = 1;
-
-  message RegionUpdateInfo {
-    required RegionInfo region = 1;
-    repeated ServerName favored_nodes = 2;
-  }
-}
-
-message UpdateFavoredNodesResponse {
-  optional uint32 response = 1;
-}
-
-/**
- * Merges the specified regions.
- * <p>
- * This method currently closes the regions and then merges them
- */
-message MergeRegionsRequest {
-  required RegionSpecifier region_a = 1;
-  required RegionSpecifier region_b = 2;
-  optional bool forcible = 3 [default = false];
-  // wall clock time from master
-  optional uint64 master_system_time = 4;
-}
-
-message MergeRegionsResponse {
-}
-
-// Protocol buffer version of WAL for replication
-message WALEntry {
-  required WALKey key = 1;
-  // Following may be null if the KVs/Cells are carried along the side in a cellblock (See
-  // RPC for more on cellblocks). If Cells/KVs are in a cellblock, this next field is null
-  // and associated_cell_count has count of Cells associated w/ this WALEntry
-  repeated bytes key_value_bytes = 2;
-  // If Cell data is carried alongside in a cellblock, this is count of Cells in the cellblock.
-  optional int32 associated_cell_count = 3;
-}
-
-/**
- * Replicates the given entries. The guarantee is that the given entries
- * will be durable on the slave cluster if this method returns without
- * any exception.  hbase.replication has to be set to true for this to work.
- */
-message ReplicateWALEntryRequest {
-  repeated WALEntry entry = 1;
-  optional string replicationClusterId = 2;
-  optional string sourceBaseNamespaceDirPath = 3;
-  optional string sourceHFileArchiveDirPath = 4;
-}
-
-message ReplicateWALEntryResponse {
-}
-
-message RollWALWriterRequest {
-}
-
-/*
- * Roll request responses no longer include regions to flush
- * this list will always be empty when talking to a 1.0 server
- */
-message RollWALWriterResponse {
-  // A list of encoded name of regions to flush
-  repeated bytes region_to_flush = 1;
-}
-
-message StopServerRequest {
-  required string reason = 1;
-}
-
-message StopServerResponse {
-}
-
-message GetServerInfoRequest {
-}
-
-message ServerInfo {
-  required ServerName server_name = 1;
-  optional uint32 webui_port = 2;
-}
-
-message GetServerInfoResponse {
-  required ServerInfo server_info = 1;
-}
-
-message UpdateConfigurationRequest {
-}
-
-message UpdateConfigurationResponse {
-}
-
-service AdminService {
-  rpc GetRegionInfo(GetRegionInfoRequest)
-    returns(GetRegionInfoResponse);
-
-  rpc GetStoreFile(GetStoreFileRequest)
-    returns(GetStoreFileResponse);
-
-  rpc GetOnlineRegion(GetOnlineRegionRequest)
-    returns(GetOnlineRegionResponse);
-
-  rpc OpenRegion(OpenRegionRequest)
-    returns(OpenRegionResponse);
-
-  rpc WarmupRegion(WarmupRegionRequest)
-    returns(WarmupRegionResponse);
-
-  rpc CloseRegion(CloseRegionRequest)
-    returns(CloseRegionResponse);
-
-  rpc FlushRegion(FlushRegionRequest)
-    returns(FlushRegionResponse);
-
-  rpc SplitRegion(SplitRegionRequest)
-    returns(SplitRegionResponse);
-
-  rpc CompactRegion(CompactRegionRequest)
-    returns(CompactRegionResponse);
-
-  rpc MergeRegions(MergeRegionsRequest)
-    returns(MergeRegionsResponse);
-
-  rpc ReplicateWALEntry(ReplicateWALEntryRequest)
-    returns(ReplicateWALEntryResponse);
-
-  rpc Replay(ReplicateWALEntryRequest)
-    returns(ReplicateWALEntryResponse);
-
-  rpc RollWALWriter(RollWALWriterRequest)
-    returns(RollWALWriterResponse);
-
-  rpc GetServerInfo(GetServerInfoRequest)
-    returns(GetServerInfoResponse);
-
-  rpc StopServer(StopServerRequest)
-    returns(StopServerResponse);
-
-  rpc UpdateFavoredNodes(UpdateFavoredNodesRequest)
-    returns(UpdateFavoredNodesResponse);
-
-  rpc UpdateConfiguration(UpdateConfigurationRequest)
-    returns(UpdateConfigurationResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Aggregate.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Aggregate.proto b/hbase-native-client/if/Aggregate.proto
deleted file mode 100644
index 4d32e70..0000000
--- a/hbase-native-client/if/Aggregate.proto
+++ /dev/null
@@ -1,63 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AggregateProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "Client.proto";
-
-message AggregateRequest {
-  /** The request passed to the AggregateService consists of three parts
-   *  (1) the (canonical) classname of the ColumnInterpreter implementation
-   *  (2) the Scan query
-   *  (3) any bytes required to construct the ColumnInterpreter object
-   *      properly
-   */
-  required string interpreter_class_name = 1;
-  required Scan scan = 2;
-  optional bytes  interpreter_specific_bytes = 3;
-}
-
-message AggregateResponse {
-  /**
-   * The AggregateService methods all have a response that either is a Pair
-   * or a simple object. When it is a Pair both first_part and second_part
-   * have defined values (and the second_part is not present in the response
-   * when the response is not a pair). Refer to the AggregateImplementation 
-   * class for an overview of the AggregateResponse object constructions. 
-   */ 
-  repeated bytes first_part = 1;
-  optional bytes second_part = 2;
-}
-
-/** Refer to the AggregateImplementation class for an overview of the 
- *  AggregateService method implementations and their functionality.
- */
-service AggregateService {
-  rpc GetMax (AggregateRequest) returns (AggregateResponse);
-  rpc GetMin (AggregateRequest) returns (AggregateResponse);
-  rpc GetSum (AggregateRequest) returns (AggregateResponse);
-  rpc GetRowNum (AggregateRequest) returns (AggregateResponse);
-  rpc GetAvg (AggregateRequest) returns (AggregateResponse);
-  rpc GetStd (AggregateRequest) returns (AggregateResponse);
-  rpc GetMedian (AggregateRequest) returns (AggregateResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Authentication.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Authentication.proto b/hbase-native-client/if/Authentication.proto
deleted file mode 100644
index 2f64799..0000000
--- a/hbase-native-client/if/Authentication.proto
+++ /dev/null
@@ -1,82 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "AuthenticationProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message AuthenticationKey {
-    required int32 id = 1;
-    required int64 expiration_date = 2;
-    required bytes key = 3;
-}
-
-
-message TokenIdentifier {
-    enum Kind {
-        HBASE_AUTH_TOKEN = 0;
-    }
-    required Kind kind = 1;
-    required bytes username = 2;
-    required int32 key_id = 3;
-    optional int64 issue_date = 4;
-    optional int64 expiration_date = 5;
-    optional int64 sequence_number = 6;
-}
-
-
-// Serialization of the org.apache.hadoop.security.token.Token class
-// Note that this is a Hadoop class, so fields may change!
-message Token {
-    // the TokenIdentifier in serialized form
-    // Note: we can't use the protobuf directly because the Hadoop Token class
-    // only stores the serialized bytes
-    optional bytes identifier = 1;
-    optional bytes password = 2;
-    optional bytes service = 3;
-}
-
-
-// RPC request & response messages
-message GetAuthenticationTokenRequest {
-}
-
-message GetAuthenticationTokenResponse {
-    optional Token token = 1;
-}
-
-message WhoAmIRequest {
-}
-
-message WhoAmIResponse {
-    optional string username = 1;
-    optional string auth_method = 2;
-}
-
-
-// RPC service
-service AuthenticationService {
-    rpc GetAuthenticationToken(GetAuthenticationTokenRequest)
-        returns (GetAuthenticationTokenResponse);
-
-    rpc WhoAmI(WhoAmIRequest)
-        returns (WhoAmIResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/BUCK b/hbase-native-client/if/BUCK
deleted file mode 100644
index 1a9721d..0000000
--- a/hbase-native-client/if/BUCK
+++ /dev/null
@@ -1,48 +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.
-
-PROTO_SRCS = glob(['*.proto'])
-HEADER_FILENAMES = [x.replace('.proto', '.pb.h') for x in PROTO_SRCS]
-CC_FILENAMES = [x.replace('.proto', '.pb.cc') for x in PROTO_SRCS]
-
-genrule(
-    name='generate-proto-sources',
-    srcs=PROTO_SRCS,
-    cmd='mkdir -p $OUT && pwd && protoc --proto_path=. --cpp_out=$OUT *.proto',
-    out='output', )
-
-for header_filename in HEADER_FILENAMES:
-    genrule(name=header_filename,
-            cmd='mkdir -p `dirname $OUT` '
-            ' && cp $(location :generate-proto-sources)/{} $OUT'.format(
-                header_filename),
-            out=header_filename, )
-for cc_filename in CC_FILENAMES:
-    genrule(
-        name=cc_filename,
-        cmd='mkdir -p `dirname $OUT` '
-        ' && cp $(location :generate-proto-sources)/*.cc `dirname $OUT` '
-        ' && cp $(location :generate-proto-sources)/*.h `dirname $OUT`'.format(
-            cc_filename),
-        out=cc_filename, )
-
-cxx_library(name='if',
-            exported_headers=[':' + x for x in HEADER_FILENAMES],
-            srcs=[':' + x for x in CC_FILENAMES],
-            deps=['//third-party:protobuf'],
-            visibility=['PUBLIC', ],
-            exported_deps=['//third-party:protobuf'])

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Cell.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Cell.proto b/hbase-native-client/if/Cell.proto
deleted file mode 100644
index 2c61035..0000000
--- a/hbase-native-client/if/Cell.proto
+++ /dev/null
@@ -1,67 +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.
- */
-
-// Cell and KeyValue protos
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "CellProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * The type of the key in a Cell
- */
-enum CellType {
-    MINIMUM = 0;
-    PUT = 4;
-
-    DELETE = 8;
-    DELETE_COLUMN = 12;
-    DELETE_FAMILY = 14;
-
-    // MAXIMUM is used when searching; you look from maximum on down.
-    MAXIMUM = 255;
-}
-
-/**
- * Protocol buffer version of Cell.
- */
-message Cell {
-  optional bytes row = 1;
-  optional bytes family = 2;
-  optional bytes qualifier = 3;
-  optional uint64 timestamp = 4;
-  optional CellType cell_type = 5;
-  optional bytes value = 6;
-  optional bytes tags = 7;
-}
-
-/**
- * Protocol buffer version of KeyValue.
- * It doesn't have those transient parameters
- */
-message KeyValue {
-  required bytes row = 1;
-  required bytes family = 2;
-  required bytes qualifier = 3;
-  optional uint64 timestamp = 4;
-  optional CellType key_type = 5;
-  optional bytes value = 6;
-  optional bytes tags = 7;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Client.proto b/hbase-native-client/if/Client.proto
deleted file mode 100644
index 8a4d459..0000000
--- a/hbase-native-client/if/Client.proto
+++ /dev/null
@@ -1,478 +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.
- */
-
-// This file contains protocol buffers that are used for Client service.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClientProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Filter.proto";
-import "Cell.proto";
-import "Comparator.proto";
-import "MapReduce.proto";
-
-/**
- * The protocol buffer version of Authorizations.
- */
-message Authorizations {
-  repeated string label = 1;
-}
-
-/**
- * The protocol buffer version of CellVisibility.
- */
-message CellVisibility {
-  required string expression = 1;
-}
-
-/**
- * Container for a list of column qualifier names of a family.
- */
-message Column {
-  required bytes family = 1;
-  repeated bytes qualifier = 2;
-}
-
-/**
- * Consistency defines the expected consistency level for an operation.
- */
-enum Consistency {
-  STRONG   = 0;
-  TIMELINE = 1;
-}
-
-/**
- * The protocol buffer version of Get.
- * Unless existence_only is specified, return all the requested data
- * for the row that matches exactly.
- */
-message Get {
-  required bytes row = 1;
-  repeated Column column = 2;
-  repeated NameBytesPair attribute = 3;
-  optional Filter filter = 4;
-  optional TimeRange time_range = 5;
-  optional uint32 max_versions = 6 [default = 1];
-  optional bool cache_blocks = 7 [default = true];
-  optional uint32 store_limit = 8;
-  optional uint32 store_offset = 9;
-
-  // The result isn't asked for, just check for
-  // the existence.
-  optional bool existence_only = 10 [default = false];
-
-  optional Consistency consistency = 12 [default = STRONG];
-  repeated ColumnFamilyTimeRange cf_time_range = 13;
-}
-
-message Result {
-  // Result includes the Cells or else it just has a count of Cells
-  // that are carried otherwise.
-  repeated Cell cell = 1;
-  // The below count is set when the associated cells are
-  // not part of this protobuf message; they are passed alongside
-  // and then this Message is just a placeholder with metadata.
-  // The count is needed to know how many to peel off the block of Cells as
-  // ours.  NOTE: This is different from the pb managed cell_count of the
-  // 'cell' field above which is non-null when the cells are pb'd.
-  optional int32 associated_cell_count = 2;
-
-  // used for Get to check existence only. Not set if existence_only was not set to true
-  //  in the query.
-  optional bool exists = 3;
-
-  // Whether or not the results are coming from possibly stale data 
-  optional bool stale = 4 [default = false];
-
-  // Whether or not the entire result could be returned. Results will be split when
-  // the RPC chunk size limit is reached. Partial results contain only a subset of the
-  // cells for a row and must be combined with a result containing the remaining cells
-  // to form a complete result
-  optional bool partial = 5 [default = false];
-}
-
-/**
- * The get request. Perform a single Get operation.
- */
-message GetRequest {
-  required RegionSpecifier region = 1;
-  required Get get = 2;
-}
-
-message GetResponse {
-  optional Result result = 1;
-}
-
-/**
- * Condition to check if the value of a given cell (row,
- * family, qualifier) matches a value via a given comparator.
- *
- * Condition is used in check and mutate operations.
- */
-message Condition {
-  required bytes row = 1;
-  required bytes family = 2;
-  required bytes qualifier = 3;
-  required CompareType compare_type = 4;
-  required Comparator comparator = 5;
-}
-
-
-/**
- * A specific mutation inside a mutate request.
- * It can be an append, increment, put or delete based
- * on the mutation type.  It can be fully filled in or
- * only metadata present because data is being carried
- * elsewhere outside of pb.
- */
-message MutationProto {
-  optional bytes row = 1;
-  optional MutationType mutate_type = 2;
-  repeated ColumnValue column_value = 3;
-  optional uint64 timestamp = 4;
-  repeated NameBytesPair attribute = 5;
-  optional Durability durability = 6 [default = USE_DEFAULT];
-
-  // For some mutations, a result may be returned, in which case,
-  // time range can be specified for potential performance gain
-  optional TimeRange time_range = 7;
-  // The below count is set when the associated cells are NOT
-  // part of this protobuf message; they are passed alongside
-  // and then this Message is a placeholder with metadata.  The
-  // count is needed to know how many to peel off the block of Cells as
-  // ours.  NOTE: This is different from the pb managed cell_count of the
-  // 'cell' field above which is non-null when the cells are pb'd.
-  optional int32 associated_cell_count = 8;
-
-  optional uint64 nonce = 9;
-
-  enum Durability {
-    USE_DEFAULT  = 0;
-    SKIP_WAL     = 1;
-    ASYNC_WAL    = 2;
-    SYNC_WAL     = 3;
-    FSYNC_WAL    = 4;
-  }
-
-  enum MutationType {
-    APPEND = 0;
-    INCREMENT = 1;
-    PUT = 2;
-    DELETE = 3;
-  }
-
-  enum DeleteType {
-    DELETE_ONE_VERSION = 0;
-    DELETE_MULTIPLE_VERSIONS = 1;
-    DELETE_FAMILY = 2;
-    DELETE_FAMILY_VERSION = 3;
-  }
-
-  message ColumnValue {
-    required bytes family = 1;
-    repeated QualifierValue qualifier_value = 2;
-
-    message QualifierValue {
-      optional bytes qualifier = 1;
-      optional bytes value = 2;
-      optional uint64 timestamp = 3;
-      optional DeleteType delete_type = 4;
-      optional bytes tags = 5;
-    }
-  }
-}
-
-/**
- * The mutate request. Perform a single Mutate operation.
- *
- * Optionally, you can specify a condition. The mutate
- * will take place only if the condition is met.  Otherwise,
- * the mutate will be ignored.  In the response result,
- * parameter processed is used to indicate if the mutate
- * actually happened.
- */
-message MutateRequest {
-  required RegionSpecifier region = 1;
-  required MutationProto mutation = 2;
-  optional Condition condition = 3;
-  optional uint64 nonce_group = 4;
-}
-
-message MutateResponse {
-  optional Result result = 1;
-
-  // used for mutate to indicate processed only
-  optional bool processed = 2;
-}
-
-/**
- * Instead of get from a table, you can scan it with optional filters.
- * You can specify the row key range, time range, the columns/families
- * to scan and so on.
- *
- * This scan is used the first time in a scan request. The response of
- * the initial scan will return a scanner id, which should be used to
- * fetch result batches later on before it is closed.
- */
-message Scan {
-  repeated Column column = 1;
-  repeated NameBytesPair attribute = 2;
-  optional bytes start_row = 3;
-  optional bytes stop_row = 4;
-  optional Filter filter = 5;
-  optional TimeRange time_range = 6;
-  optional uint32 max_versions = 7 [default = 1];
-  optional bool cache_blocks = 8 [default = true];
-  optional uint32 batch_size = 9;
-  optional uint64 max_result_size = 10;
-  optional uint32 store_limit = 11;
-  optional uint32 store_offset = 12;
-  optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */
-  optional bool small = 14;
-  optional bool reversed = 15 [default = false];
-  optional Consistency consistency = 16 [default = STRONG];
-  optional uint32 caching = 17;
-  optional bool allow_partial_results = 18;
-  repeated ColumnFamilyTimeRange cf_time_range = 19;
-}
-
-/**
- * A scan request. Initially, it should specify a scan. Later on, you
- * can use the scanner id returned to fetch result batches with a different
- * scan request.
- *
- * The scanner will remain open if there are more results, and it's not
- * asked to be closed explicitly.
- *
- * You can fetch the results and ask the scanner to be closed to save
- * a trip if you are not interested in remaining results.
- */
-message ScanRequest {
-  optional RegionSpecifier region = 1;
-  optional Scan scan = 2;
-  optional uint64 scanner_id = 3;
-  optional uint32 number_of_rows = 4;
-  optional bool close_scanner = 5;
-  optional uint64 next_call_seq = 6;
-  optional bool client_handles_partials = 7;
-  optional bool client_handles_heartbeats = 8;
-  optional bool track_scan_metrics = 9;
-  optional bool renew = 10 [default = false];
-}
-
-/**
- * The scan response. If there are no more results, more_results will
- * be false.  If it is not specified, it means there are more.
- */
-message ScanResponse {
-  // This field is filled in if we are doing cellblocks.  A cellblock is made up
-  // of all Cells serialized out as one cellblock BUT responses from a server
-  // have their Cells grouped by Result.  So we can reconstitute the
-  // Results on the client-side, this field is a list of counts of Cells
-  // in each Result that makes up the response.  For example, if this field
-  // has 3, 3, 3 in it, then we know that on the client, we are to make
-  // three Results each of three Cells each.
-  repeated uint32 cells_per_result = 1;
-
-  optional uint64 scanner_id = 2;
-  optional bool more_results = 3;
-  optional uint32 ttl = 4;
-  // If cells are not carried in an accompanying cellblock, then they are pb'd here.
-  // This field is mutually exclusive with cells_per_result (since the Cells will
-  // be inside the pb'd Result)
-  repeated Result results = 5;
-  optional bool stale = 6;
-
-  // This field is filled in if we are doing cellblocks. In the event that a row
-  // could not fit all of its cells into a single RPC chunk, the results will be
-  // returned as partials, and reconstructed into a complete result on the client
-  // side. This field is a list of flags indicating whether or not the result
-  // that the cells belong to is a partial result. For example, if this field
-  // has false, false, true in it, then we know that on the client side, we need to
-  // make another RPC request since the last result was only a partial.
-  repeated bool partial_flag_per_result = 7;
-
-  // A server may choose to limit the number of results returned to the client for
-  // reasons such as the size in bytes or quantity of results accumulated. This field
-  // will true when more results exist in the current region.
-  optional bool more_results_in_region = 8;
-  
-  // This field is filled in if the server is sending back a heartbeat message.
-  // Heartbeat messages are sent back to the client to prevent the scanner from
-  // timing out. Seeing a heartbeat message communicates to the Client that the
-  // server would have continued to scan had the time limit not been reached.
-  optional bool heartbeat_message = 9;
-  
-  // This field is filled in if the client has requested that scan metrics be tracked.
-  // The metrics tracked here are sent back to the client to be tracked together with 
-  // the existing client side metrics.
-  optional ScanMetrics scan_metrics = 10;
-}
-
-/**
- * Atomically bulk load multiple HFiles (say from different column families)
- * into an open region.
- */
-message BulkLoadHFileRequest {
-  required RegionSpecifier region = 1;
-  repeated FamilyPath family_path = 2;
-  optional bool assign_seq_num = 3;
-
-  message FamilyPath {
-    required bytes family = 1;
-    required string path = 2;
-  }
-}
-
-message BulkLoadHFileResponse {
-  required bool loaded = 1;
-}
-
-message CoprocessorServiceCall {
-  required bytes row = 1;
-  required string service_name = 2;
-  required string method_name = 3;
-  required bytes request = 4;
-}
-
-message CoprocessorServiceResult {
-  optional NameBytesPair value = 1;
-}
-
-message CoprocessorServiceRequest {
-  required RegionSpecifier region = 1;
-  required CoprocessorServiceCall call = 2;
-}
-
-message CoprocessorServiceResponse {
-  required RegionSpecifier region = 1;
-  required NameBytesPair value = 2;
-}
-
-// Either a Get or a Mutation
-message Action {
-  // If part of a multi action, useful aligning
-  // result with what was originally submitted.
-  optional uint32 index = 1;
-  optional MutationProto mutation = 2;
-  optional Get get = 3;
-  optional CoprocessorServiceCall service_call = 4;
-}
-
-/**
- * Actions to run against a Region.
- */
-message RegionAction {
-  required RegionSpecifier region = 1;
-  // When set, run mutations as atomic unit.
-  optional bool atomic = 2;
-  repeated Action action = 3;
-}
-
-/*
-* Statistics about the current load on the region
-*/
-message RegionLoadStats {
-  // Percent load on the memstore. Guaranteed to be positive, between 0 and 100.
-  optional int32 memstoreLoad = 1 [default = 0];
-  // Percent JVM heap occupancy. Guaranteed to be positive, between 0 and 100.
-  // We can move this to "ServerLoadStats" should we develop them.
-  optional int32 heapOccupancy = 2 [default = 0];
-  // Compaction pressure. Guaranteed to be positive, between 0 and 100.
-  optional int32 compactionPressure = 3 [default = 0];
-}
-
-message MultiRegionLoadStats{
-  repeated RegionSpecifier region = 1;
-  repeated RegionLoadStats stat = 2;
-}
-
-/**
- * Either a Result or an Exception NameBytesPair (keyed by
- * exception name whose value is the exception stringified)
- * or maybe empty if no result and no exception.
- */
-message ResultOrException {
-  // If part of a multi call, save original index of the list of all
-  // passed so can align this response w/ original request.
-  optional uint32 index = 1;
-  optional Result result = 2;
-  optional NameBytesPair exception = 3;
-  // result if this was a coprocessor service call
-  optional CoprocessorServiceResult service_result = 4;
-  // current load on the region
-  optional RegionLoadStats loadStats = 5 [deprecated=true];
-}
-
-/**
- * The result of a RegionAction.
- */
-message RegionActionResult {
-  repeated ResultOrException resultOrException = 1;
-  // If the operation failed globally for this region, this exception is set
-  optional NameBytesPair exception = 2;
-}
-
-/**
- * Execute a list of actions on a given region in order.
- * Nothing prevents a request to contains a set of RegionAction on the same region.
- * For this reason, the matching between the MultiRequest and the MultiResponse is not
- *  done by the region specifier but by keeping the order of the RegionActionResult vs.
- *  the order of the RegionAction.
- */
-message MultiRequest {
-  repeated RegionAction regionAction = 1;
-  optional uint64 nonceGroup = 2;
-  optional Condition condition = 3;
-}
-
-message MultiResponse {
-  repeated RegionActionResult regionActionResult = 1;
-  // used for mutate to indicate processed only
-  optional bool processed = 2;
-  optional MultiRegionLoadStats regionStatistics = 3;
-}
-
-
-service ClientService {
-  rpc Get(GetRequest)
-    returns(GetResponse);
-
-  rpc Mutate(MutateRequest)
-    returns(MutateResponse);
-
-  rpc Scan(ScanRequest)
-    returns(ScanResponse);
-
-  rpc BulkLoadHFile(BulkLoadHFileRequest)
-    returns(BulkLoadHFileResponse);
-
-  rpc ExecService(CoprocessorServiceRequest)
-    returns(CoprocessorServiceResponse);
-    
-  rpc ExecRegionServerService(CoprocessorServiceRequest)
-    returns(CoprocessorServiceResponse);
-
-  rpc Multi(MultiRequest)
-    returns(MultiResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/ClusterId.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/ClusterId.proto b/hbase-native-client/if/ClusterId.proto
deleted file mode 100644
index aed8cfc..0000000
--- a/hbase-native-client/if/ClusterId.proto
+++ /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.
- */
-
-// This file contains protocol buffers that are shared throughout HBase
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClusterIdProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * Content of the '/hbase/hbaseid', cluster id, znode.
- * Also cluster of the ${HBASE_ROOTDIR}/hbase.id file.
- */
-message ClusterId {
-  // This is the cluster id, a uuid as a String
-  required string cluster_id = 1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/ClusterStatus.proto b/hbase-native-client/if/ClusterStatus.proto
deleted file mode 100644
index 54bc0c3..0000000
--- a/hbase-native-client/if/ClusterStatus.proto
+++ /dev/null
@@ -1,227 +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.
- */
-
-// This file contains protocol buffers that are used for ClustStatus
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ClusterStatusProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "ClusterId.proto";
-import "FS.proto";
-
-message RegionState {
-  required RegionInfo region_info = 1;
-  required State state = 2;
-  optional uint64 stamp = 3;
-  enum State {
-    OFFLINE = 0;       // region is in an offline state
-    PENDING_OPEN = 1;  // sent rpc to server to open but has not begun
-    OPENING = 2;       // server has begun to open but not yet done
-    OPEN = 3;          // server opened region and updated meta
-    PENDING_CLOSE = 4; // sent rpc to server to close but has not begun
-    CLOSING = 5;       // server has begun to close but not yet done
-    CLOSED = 6;        // server closed region and updated meta
-    SPLITTING = 7;     // server started split of a region
-    SPLIT = 8;         // server completed split of a region
-    FAILED_OPEN = 9;   // failed to open, and won't retry any more
-    FAILED_CLOSE = 10; // failed to close, and won't retry any more
-    MERGING = 11;      // server started merge a region
-    MERGED = 12;       // server completed merge of a region
-    SPLITTING_NEW = 13;  // new region to be created when RS splits a parent
-                       // region but hasn't be created yet, or master doesn't
-                       // know it's already created
-    MERGING_NEW = 14;  // new region to be created when RS merges two
-                       // daughter regions but hasn't be created yet, or
-                       // master doesn't know it's already created
-  }
-}
-
-message RegionInTransition {
-  required RegionSpecifier spec = 1;
-  required RegionState region_state = 2;
-}
-
-/**
- * sequence Id of a store
- */
-message StoreSequenceId {
-  required bytes family_name = 1;
-  required uint64 sequence_id = 2;
-}
-
-/**
- * contains a sequence id of a region which should be the minimum of its store sequence ids and
- * list of sequence ids of the region's stores
- */
-message RegionStoreSequenceIds {
-  required uint64 last_flushed_sequence_id = 1;
-  repeated StoreSequenceId store_sequence_id = 2;
-}
-
-message RegionLoad {
-  /** the region specifier */
-  required RegionSpecifier region_specifier = 1;
-
-  /** the number of stores for the region */
-  optional uint32 stores = 2;
-
-  /** the number of storefiles for the region */
-  optional uint32 storefiles = 3;
-
-  /** the total size of the store files for the region, uncompressed, in MB */
-  optional uint32 store_uncompressed_size_MB = 4;
-
-  /** the current total size of the store files for the region, in MB */
-  optional uint32 storefile_size_MB = 5;
-
-  /** the current size of the memstore for the region, in MB */
-  optional uint32 memstore_size_MB = 6;
-
-  /**
-   * The current total size of root-level store file indexes for the region,
-   * in MB. The same as {@link #rootIndexSizeKB} but in MB.
-   */
-  optional uint32 storefile_index_size_MB = 7;
-
-  /** the current total read requests made to region */
-  optional uint64 read_requests_count = 8;
-
-  /** the current total write requests made to region */
-  optional uint64 write_requests_count = 9;
-
-  /** the total compacting key values in currently running compaction */
-  optional uint64 total_compacting_KVs = 10;
-
-  /** the completed count of key values in currently running compaction */
-  optional uint64 current_compacted_KVs = 11;
-
-   /** The current total size of root-level indexes for the region, in KB. */
-  optional uint32 root_index_size_KB = 12;
-
-  /** The total size of all index blocks, not just the root level, in KB. */
-  optional uint32 total_static_index_size_KB = 13;
-
-  /**
-   * The total size of all Bloom filter blocks, not just loaded into the
-   * block cache, in KB.
-   */
-  optional uint32 total_static_bloom_size_KB = 14;
-
-  /** the most recent sequence Id from cache flush */
-  optional uint64 complete_sequence_id = 15;
-
-  /** The current data locality for region in the regionserver */
-  optional float data_locality = 16;
-
-  optional uint64 last_major_compaction_ts = 17 [default = 0];
-
-  /** the most recent sequence Id of store from cache flush */
-  repeated StoreSequenceId store_complete_sequence_id = 18;
-
-  /** the current total filtered read requests made to region */
-  optional uint64 filtered_read_requests_count = 19;
-}
-
-/* Server-level protobufs */
-
-message ReplicationLoadSink {
-  required uint64 ageOfLastAppliedOp = 1;
-  required uint64 timeStampsOfLastAppliedOp = 2;
-}
-
-message ReplicationLoadSource {
-  required string peerID = 1;
-  required uint64 ageOfLastShippedOp = 2;
-  required uint32 sizeOfLogQueue = 3;
-  required uint64 timeStampOfLastShippedOp = 4;
-  required uint64 replicationLag = 5;
-}
-
-message ServerLoad {
-  /** Number of requests since last report. */
-  optional uint64 number_of_requests = 1;
-
-  /** Total Number of requests from the start of the region server. */
-  optional uint64 total_number_of_requests = 2;
-
-  /** the amount of used heap, in MB. */
-  optional uint32 used_heap_MB = 3;
-
-  /** the maximum allowable size of the heap, in MB. */
-  optional uint32 max_heap_MB = 4;
-
-  /** Information on the load of individual regions. */
-  repeated RegionLoad region_loads = 5;
-
-  /**
-   * Regionserver-level coprocessors, e.g., WALObserver implementations.
-   * Region-level coprocessors, on the other hand, are stored inside RegionLoad
-   * objects.
-   */
-  repeated Coprocessor coprocessors = 6;
-
-  /**
-   * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
-   * time is measured as the difference, measured in milliseconds, between the current time
-   * and midnight, January 1, 1970 UTC.
-   */
-  optional uint64 report_start_time = 7;
-
-  /**
-   * Time when report was generated.
-   * time is measured as the difference, measured in milliseconds, between the current time
-   * and midnight, January 1, 1970 UTC.
-   */
-  optional uint64 report_end_time = 8;
-
-  /**
-   * The port number that this region server is hosing an info server on.
-   */
-  optional uint32 info_server_port = 9;
-
-  /**
-   * The replicationLoadSource for the replication Source status of this region server.
-   */
-  repeated ReplicationLoadSource replLoadSource = 10;
-
-  /**
-   * The replicationLoadSink for the replication Sink status of this region server.
-   */
-  optional ReplicationLoadSink replLoadSink = 11;
-}
-
-message LiveServerInfo {
-  required ServerName server = 1;
-  required ServerLoad server_load = 2;
-}
-
-message ClusterStatus {
-  optional HBaseVersionFileContent hbase_version = 1;
-  repeated LiveServerInfo live_servers = 2;
-  repeated ServerName dead_servers = 3;
-  repeated RegionInTransition regions_in_transition = 4;
-  optional ClusterId cluster_id = 5;
-  repeated Coprocessor master_coprocessors = 6;
-  optional ServerName master = 7;
-  repeated ServerName backup_masters = 8;
-  optional bool balancer_on = 9;
-}


[24/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/request.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/request.cc b/hbase-native-client/connection/request.cc
deleted file mode 100644
index 8983726..0000000
--- a/hbase-native-client/connection/request.cc
+++ /dev/null
@@ -1,46 +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 "connection/request.h"
-
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-Request::Request(std::shared_ptr<google::protobuf::Message> req,
-                 std::shared_ptr<google::protobuf::Message> resp, std::string method)
-    : req_msg_(req), resp_msg_(resp), method_(method), call_id_(0) {}
-
-std::unique_ptr<Request> Request::get() {
-  return std::make_unique<Request>(std::make_shared<hbase::pb::GetRequest>(),
-                                   std::make_shared<hbase::pb::GetResponse>(), "Get");
-}
-std::unique_ptr<Request> Request::mutate() {
-  return std::make_unique<Request>(std::make_shared<hbase::pb::MutateRequest>(),
-                                   std::make_shared<hbase::pb::MutateResponse>(), "Mutate");
-}
-std::unique_ptr<Request> Request::scan() {
-  return std::make_unique<Request>(std::make_shared<hbase::pb::ScanRequest>(),
-                                   std::make_shared<hbase::pb::ScanResponse>(), "Scan");
-}
-std::unique_ptr<Request> Request::multi() {
-  return std::make_unique<Request>(std::make_shared<hbase::pb::MultiRequest>(),
-                                   std::make_shared<hbase::pb::MultiResponse>(), "Multi");
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/request.h b/hbase-native-client/connection/request.h
deleted file mode 100644
index 4b652c0..0000000
--- a/hbase-native-client/connection/request.h
+++ /dev/null
@@ -1,79 +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.
- *
- */
-#pragma once
-
-#include <folly/Conv.h>
-#include <google/protobuf/message.h>
-
-#include <cstdint>
-#include <memory>
-#include <string>
-
-namespace hbase {
-
-/**
- * Main request class.
- * This holds the request object and the un-filled in approriatley typed
- * response object.
- */
-class Request {
- public:
-  /** Create a request object for a get */
-  static std::unique_ptr<Request> get();
-  /** Create a request object for a mutate */
-  static std::unique_ptr<Request> mutate();
-  /** Create a request object for a scan */
-  static std::unique_ptr<Request> scan();
-  /** Create a request object for a multi */
-  static std::unique_ptr<Request> multi();
-
-  /**
-   * This should be private. Do not use this.
-   *
-   *
-   * Constructor that's public for make_unique. This sets all the messages and
-   * method name.
-   */
-  Request(std::shared_ptr<google::protobuf::Message> req,
-          std::shared_ptr<google::protobuf::Message> resp, std::string method);
-
-  /** Get the call id. */
-  uint32_t call_id() { return call_id_; }
-  /** Set the call id. This should only be set once. */
-  void set_call_id(uint32_t call_id) { call_id_ = call_id; }
-  /** Get the backing request protobuf message. */
-  std::shared_ptr<google::protobuf::Message> req_msg() { return req_msg_; }
-  /** Get the backing response protobuf message. */
-  std::shared_ptr<google::protobuf::Message> resp_msg() { return resp_msg_; }
-  /** Get the method name. This is used to the the receiving rpc server what
-   * method type to decode. */
-  std::string method() { return method_; }
-
-  std::string DebugString() {
-    return "call_id:" + folly::to<std::string>(call_id_) + ", req_msg:" +
-           req_msg_->ShortDebugString() + ", method:" + method_;
-  }
-
- private:
-  uint32_t call_id_;
-  std::shared_ptr<google::protobuf::Message> req_msg_ = nullptr;
-  std::shared_ptr<google::protobuf::Message> resp_msg_ = nullptr;
-  std::string method_ = "Get";
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/response.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/response.h b/hbase-native-client/connection/response.h
deleted file mode 100644
index 38fdda0..0000000
--- a/hbase-native-client/connection/response.h
+++ /dev/null
@@ -1,93 +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.
- *
- */
-#pragma once
-
-#include <folly/Conv.h>
-#include <folly/ExceptionWrapper.h>
-
-#include <cstdint>
-#include <memory>
-#include <string>
-#include <utility>
-
-#include "serde/cell-scanner.h"
-
-// Forward
-namespace google {
-namespace protobuf {
-class Message;
-}
-}
-
-namespace hbase {
-
-/**
- * @brief Class representing a rpc response
- *
- * This is the class sent to a service.
- */
-class Response {
- public:
-  /**
-   * Constructor.
-   * Initinalizes the call id to 0. 0 should never be a valid call id.
-   */
-  Response() : call_id_(0), resp_msg_(nullptr), cell_scanner_(nullptr), exception_(nullptr) {}
-
-  /** Get the call_id */
-  uint32_t call_id() { return call_id_; }
-
-  /** Set the call_id */
-  void set_call_id(uint32_t call_id) { call_id_ = call_id; }
-
-  /**
-   * Get the response message.
-   * The caller is reponsible for knowing the type. In practice the call id is
-   * used to figure out the type.
-   */
-  std::shared_ptr<google::protobuf::Message> resp_msg() const { return resp_msg_; }
-
-  /** Set the response message. */
-  void set_resp_msg(std::shared_ptr<google::protobuf::Message> response) {
-    resp_msg_ = std::move(response);
-  }
-
-  void set_cell_scanner(std::shared_ptr<CellScanner> cell_scanner) { cell_scanner_ = cell_scanner; }
-
-  const std::shared_ptr<CellScanner> cell_scanner() const { return cell_scanner_; }
-
-  folly::exception_wrapper exception() { return exception_; }
-
-  void set_exception(folly::exception_wrapper value) { exception_ = value; }
-
-  std::string DebugString() const {
-    std::string s{"call_id:"};
-    s += folly::to<std::string>(call_id_);
-    s += ", resp_msg:";
-    s += resp_msg_->ShortDebugString();
-    return s;
-  }
-
- private:
-  uint32_t call_id_;
-  std::shared_ptr<google::protobuf::Message> resp_msg_;
-  std::shared_ptr<CellScanner> cell_scanner_;
-  folly::exception_wrapper exception_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-client.cc b/hbase-native-client/connection/rpc-client.cc
deleted file mode 100644
index 51c9c63..0000000
--- a/hbase-native-client/connection/rpc-client.cc
+++ /dev/null
@@ -1,119 +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 "connection/rpc-client.h"
-
-#include <folly/Format.h>
-#include <folly/Logging.h>
-#include <folly/futures/Future.h>
-#include <unistd.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-#include "exceptions/exception.h"
-
-using hbase::security::User;
-using std::chrono::nanoseconds;
-
-namespace hbase {
-
-RpcClient::RpcClient(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                     std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                     std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
-                     nanoseconds connect_timeout)
-    : io_executor_(io_executor), conf_(conf) {
-  cp_ = std::make_shared<ConnectionPool>(io_executor_, cpu_executor, codec, conf, connect_timeout);
-}
-
-void RpcClient::Close() { io_executor_->stop(); }
-
-std::unique_ptr<Response> RpcClient::SyncCall(const std::string& host, uint16_t port,
-                                              std::unique_ptr<Request> req,
-                                              std::shared_ptr<User> ticket) {
-  return AsyncCall(host, port, std::move(req), ticket).get();
-}
-
-std::unique_ptr<Response> RpcClient::SyncCall(const std::string& host, uint16_t port,
-                                              std::unique_ptr<Request> req,
-                                              std::shared_ptr<User> ticket,
-                                              const std::string& service_name) {
-  return AsyncCall(host, port, std::move(req), ticket, service_name).get();
-}
-
-folly::Future<std::unique_ptr<Response>> RpcClient::AsyncCall(const std::string& host,
-                                                              uint16_t port,
-                                                              std::unique_ptr<Request> req,
-                                                              std::shared_ptr<User> ticket) {
-  auto remote_id = std::make_shared<ConnectionId>(host, port, ticket);
-  return SendRequest(remote_id, std::move(req));
-}
-
-folly::Future<std::unique_ptr<Response>> RpcClient::AsyncCall(const std::string& host,
-                                                              uint16_t port,
-                                                              std::unique_ptr<Request> req,
-                                                              std::shared_ptr<User> ticket,
-                                                              const std::string& service_name) {
-  auto remote_id = std::make_shared<ConnectionId>(host, port, ticket, service_name);
-  return SendRequest(remote_id, std::move(req));
-}
-
-/**
- * There are two cases for ConnectionException:
- * 1. The first time connection
- * establishment, i.e. GetConnection(remote_id), AsyncSocketException being a cause.
- * 2. Writing request down the pipeline, i.e. RpcConnection::SendRequest, AsyncSocketException being
- * a cause as well.
- */
-folly::Future<std::unique_ptr<Response>> RpcClient::SendRequest(
-    std::shared_ptr<ConnectionId> remote_id, std::unique_ptr<Request> req) {
-  try {
-    return GetConnection(remote_id)
-        ->SendRequest(std::move(req))
-        .onError([&, this](const folly::exception_wrapper& ew) {
-          VLOG(3) << folly::sformat("RpcClient Exception: {}", ew.what());
-          ew.with_exception([&, this](const hbase::ConnectionException& re) {
-            /* bad connection, remove it from pool. */
-            cp_->Close(remote_id);
-          });
-          return GetFutureWithException(ew);
-        });
-  } catch (const ConnectionException& e) {
-    CHECK(e.cause().get_exception() != nullptr);
-    VLOG(3) << folly::sformat("RpcClient Exception: {}", e.cause().what());
-    /* bad connection, remove it from pool. */
-    cp_->Close(remote_id);
-    return GetFutureWithException(e);
-  }
-}
-
-template <typename EXCEPTION>
-folly::Future<std::unique_ptr<Response>> RpcClient::GetFutureWithException(const EXCEPTION& e) {
-  return GetFutureWithException(folly::exception_wrapper{e});
-}
-
-folly::Future<std::unique_ptr<Response>> RpcClient::GetFutureWithException(
-    const folly::exception_wrapper& ew) {
-  folly::Promise<std::unique_ptr<Response>> promise;
-  auto future = promise.getFuture();
-  promise.setException(ew);
-  return future;
-}
-
-std::shared_ptr<RpcConnection> RpcClient::GetConnection(std::shared_ptr<ConnectionId> remote_id) {
-  return cp_->GetConnection(remote_id);
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-client.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-client.h b/hbase-native-client/connection/rpc-client.h
deleted file mode 100644
index 93801d8..0000000
--- a/hbase-native-client/connection/rpc-client.h
+++ /dev/null
@@ -1,83 +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.
- *
- */
-#pragma once
-
-#include <google/protobuf/service.h>
-
-#include <folly/ExceptionWrapper.h>
-#include <chrono>
-#include <memory>
-#include <string>
-#include <utility>
-
-#include "connection/connection-id.h"
-#include "connection/connection-pool.h"
-#include "connection/request.h"
-#include "connection/response.h"
-#include "security/user.h"
-
-namespace hbase {
-
-class RpcClient {
- public:
-  RpcClient(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-            std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-            std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
-            std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
-
-  virtual ~RpcClient() { Close(); }
-
-  virtual std::unique_ptr<Response> SyncCall(const std::string &host, uint16_t port,
-                                             std::unique_ptr<Request> req,
-                                             std::shared_ptr<security::User> ticket);
-
-  virtual std::unique_ptr<Response> SyncCall(const std::string &host, uint16_t port,
-                                             std::unique_ptr<Request> req,
-                                             std::shared_ptr<security::User> ticket,
-                                             const std::string &service_name);
-
-  virtual folly::Future<std::unique_ptr<Response>> AsyncCall(
-      const std::string &host, uint16_t port, std::unique_ptr<Request> req,
-      std::shared_ptr<security::User> ticket);
-
-  virtual folly::Future<std::unique_ptr<Response>> AsyncCall(const std::string &host, uint16_t port,
-                                                             std::unique_ptr<Request> req,
-                                                             std::shared_ptr<security::User> ticket,
-                                                             const std::string &service_name);
-
-  virtual void Close();
-
-  std::shared_ptr<ConnectionPool> connection_pool() const { return cp_; }
-
- private:
-  std::shared_ptr<RpcConnection> GetConnection(std::shared_ptr<ConnectionId> remote_id);
-  folly::Future<std::unique_ptr<Response>> SendRequest(std::shared_ptr<ConnectionId> remote_id,
-                                                       std::unique_ptr<Request> req);
-  template <typename EXCEPTION>
-  folly::Future<std::unique_ptr<Response>> GetFutureWithException(const EXCEPTION &e);
-
-  folly::Future<std::unique_ptr<Response>> GetFutureWithException(
-      const folly::exception_wrapper &ew);
-
- private:
-  std::shared_ptr<ConnectionPool> cp_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<Configuration> conf_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-connection.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-connection.h b/hbase-native-client/connection/rpc-connection.h
deleted file mode 100644
index 9063280..0000000
--- a/hbase-native-client/connection/rpc-connection.h
+++ /dev/null
@@ -1,79 +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.
- *
- */
-#pragma once
-
-#include <memory>
-#include <mutex>
-#include <utility>
-
-#include "connection/connection-factory.h"
-#include "connection/connection-id.h"
-#include "connection/request.h"
-#include "connection/response.h"
-#include "connection/service.h"
-
-namespace hbase {
-
-class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
- public:
-  RpcConnection(std::shared_ptr<ConnectionId> connection_id, std::shared_ptr<ConnectionFactory> cf)
-      : connection_id_(connection_id), cf_(cf), hbase_service_(nullptr) {}
-
-  virtual ~RpcConnection() { Close(); }
-
-  virtual std::shared_ptr<ConnectionId> remote_id() const { return connection_id_; }
-
-  virtual folly::Future<std::unique_ptr<Response>> SendRequest(std::unique_ptr<Request> req) {
-    std::lock_guard<std::recursive_mutex> lock(mutex_);
-    if (hbase_service_ == nullptr) {
-      Connect();
-    }
-    VLOG(5) << "Calling RpcConnection::SendRequest()";  // TODO
-    return (*hbase_service_)(std::move(req));
-  }
-
-  virtual void Close() {
-    std::lock_guard<std::recursive_mutex> lock(mutex_);
-    if (hbase_service_) {
-      hbase_service_->close();
-      hbase_service_ = nullptr;
-    }
-    if (client_bootstrap_) {
-      client_bootstrap_ = nullptr;
-    }
-  }
-
- private:
-  void Connect() {
-    client_bootstrap_ = cf_->MakeBootstrap();
-    auto dispatcher = cf_->Connect(shared_from_this(), client_bootstrap_, remote_id()->host(),
-                                   remote_id()->port());
-    hbase_service_ = std::move(dispatcher);
-  }
-
- private:
-  std::recursive_mutex mutex_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<ConnectionId> connection_id_;
-  std::shared_ptr<HBaseService> hbase_service_;
-  std::shared_ptr<ConnectionFactory> cf_;
-  std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-fault-injector-inl.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-fault-injector-inl.h b/hbase-native-client/connection/rpc-fault-injector-inl.h
deleted file mode 100644
index 8bbaddf..0000000
--- a/hbase-native-client/connection/rpc-fault-injector-inl.h
+++ /dev/null
@@ -1,41 +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.
- *
- */
-#pragma once
-
-namespace hbase {
-
-template <typename T>
-std::shared_ptr<T> RpcFaultInjector<T>::instance = std::make_shared<T>();
-
-template <typename T>
-RpcFaultInjector<T>::RpcFaultInjector() {}
-
-template <typename T>
-RpcFaultInjector<T>::~RpcFaultInjector() {}
-
-template <typename T>
-std::shared_ptr<T> RpcFaultInjector<T>::Get() {
-  return instance;
-}
-
-template <typename T>
-void RpcFaultInjector<T>::Set(std::shared_ptr<T> injector) {
-  instance = injector;
-}
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-fault-injector.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-fault-injector.cc b/hbase-native-client/connection/rpc-fault-injector.cc
deleted file mode 100644
index 16e2034..0000000
--- a/hbase-native-client/connection/rpc-fault-injector.cc
+++ /dev/null
@@ -1,21 +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 "rpc-fault-injector.h"
-
-namespace hbase {} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-fault-injector.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-fault-injector.h b/hbase-native-client/connection/rpc-fault-injector.h
deleted file mode 100644
index 2733b7d..0000000
--- a/hbase-native-client/connection/rpc-fault-injector.h
+++ /dev/null
@@ -1,50 +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.
- *
- */
-#pragma once
-
-#include <folly/io/async/AsyncTransport.h>
-#include "connection/pipeline.h"
-
-namespace hbase {
-
-template <typename T>
-class RpcFaultInjector {
- public:
-  RpcFaultInjector();
-  virtual ~RpcFaultInjector();
-
-  static std::shared_ptr<T> Get();
-  static void Set(std::shared_ptr<T> instance);
-
- private:
-  static std::shared_ptr<T> instance;
-};
-
-class RpcClientFaultInjector : public RpcFaultInjector<RpcClientFaultInjector> {
- public:
-  RpcClientFaultInjector() {}
-  virtual ~RpcClientFaultInjector() {}
-  /**
-   * Here goes virtual functions for injecting various faults. They should be no-ops by default.
-   * Sub classes of RpcClientFaultInjector will override by providing concrete faults.
-   */
-};
-} /* namespace hbase */
-
-#include "connection/rpc-fault-injector-inl.h"

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-test-server-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-test-server-handler.cc b/hbase-native-client/connection/rpc-test-server-handler.cc
deleted file mode 100644
index 8e405ef..0000000
--- a/hbase-native-client/connection/rpc-test-server-handler.cc
+++ /dev/null
@@ -1,81 +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 "connection/rpc-test-server-handler.h"
-#include "if/RPC.pb.h"
-#include "if/test.pb.h"
-
-namespace hbase {
-
-void RpcTestServerSerializeHandler::read(Context* ctx, std::unique_ptr<folly::IOBuf> buf) {
-  buf->coalesce();
-  pb::RequestHeader header;
-
-  int used_bytes = serde_.ParseDelimited(buf.get(), &header);
-  VLOG(3) << "Read RPC RequestHeader size=" << used_bytes << " call_id=" << header.call_id();
-
-  auto received = CreateReceivedRequest(header.method_name());
-
-  buf->trimStart(used_bytes);
-  if (header.has_request_param() && received != nullptr) {
-    used_bytes = serde_.ParseDelimited(buf.get(), received->req_msg().get());
-    VLOG(3) << "Read RPCRequest, buf length:" << buf->length()
-            << ", header PB length:" << used_bytes;
-    received->set_call_id(header.call_id());
-  }
-
-  if (received != nullptr) {
-    ctx->fireRead(std::move(received));
-  }
-}
-
-folly::Future<folly::Unit> RpcTestServerSerializeHandler::write(Context* ctx,
-                                                                std::unique_ptr<Response> resp) {
-  VLOG(3) << "Writing RPC Request";
-  // Send the data down the pipeline.
-  return ctx->fireWrite(
-      serde_.Response(resp->call_id(), resp->resp_msg().get(), resp->exception()));
-}
-
-std::unique_ptr<Request> RpcTestServerSerializeHandler::CreateReceivedRequest(
-    const std::string& method_name) {
-  std::unique_ptr<Request> result = nullptr;
-
-  if (method_name == "ping") {
-    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                       std::make_shared<EmptyResponseProto>(), method_name);
-  } else if (method_name == "echo") {
-    result = std::make_unique<Request>(std::make_shared<EchoRequestProto>(),
-                                       std::make_shared<EchoResponseProto>(), method_name);
-  } else if (method_name == "error") {
-    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                       std::make_shared<EmptyResponseProto>(), method_name);
-  } else if (method_name == "pause") {
-    result = std::make_unique<Request>(std::make_shared<PauseRequestProto>(),
-                                       std::make_shared<EmptyResponseProto>(), method_name);
-  } else if (method_name == "addr") {
-    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                       std::make_shared<AddrResponseProto>(), method_name);
-  } else if (method_name == "socketNotOpen") {
-    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                       std::make_shared<EmptyResponseProto>(), method_name);
-  }
-  return result;
-}
-}  // end of namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-test-server-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-test-server-handler.h b/hbase-native-client/connection/rpc-test-server-handler.h
deleted file mode 100644
index ab0264f..0000000
--- a/hbase-native-client/connection/rpc-test-server-handler.h
+++ /dev/null
@@ -1,47 +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.
- *
- */
-#pragma once
-
-#include <wangle/channel/Handler.h>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "serde/rpc-serde.h"
-
-using namespace hbase;
-
-namespace hbase {
-// A real rpc server would probably use generated client/server stubs
-class RpcTestServerSerializeHandler
-    : public wangle::Handler<std::unique_ptr<folly::IOBuf>, std::unique_ptr<Request>,
-                             std::unique_ptr<Response>, std::unique_ptr<folly::IOBuf>> {
- public:
-  RpcTestServerSerializeHandler() : serde_() {}
-
-  void read(Context* ctx, std::unique_ptr<folly::IOBuf> buf) override;
-
-  folly::Future<folly::Unit> write(Context* ctx, std::unique_ptr<Response> resp) override;
-
- private:
-  std::unique_ptr<Request> CreateReceivedRequest(const std::string& method_name);
-
- private:
-  hbase::RpcSerde serde_;
-};
-}  // end of namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-test-server.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-test-server.cc b/hbase-native-client/connection/rpc-test-server.cc
deleted file mode 100644
index 157ea71..0000000
--- a/hbase-native-client/connection/rpc-test-server.cc
+++ /dev/null
@@ -1,108 +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 <wangle/channel/AsyncSocketHandler.h>
-#include <wangle/channel/EventBaseHandler.h>
-#include <wangle/codec/LengthFieldBasedFrameDecoder.h>
-#include <wangle/codec/LengthFieldPrepender.h>
-#include <wangle/service/ServerDispatcher.h>
-
-#include "connection/rpc-test-server-handler.h"
-#include "connection/rpc-test-server.h"
-#include "if/test.pb.h"
-
-namespace hbase {
-
-RpcTestServerSerializePipeline::Ptr RpcTestServerPipelineFactory::newPipeline(
-    std::shared_ptr<AsyncTransportWrapper> sock) {
-  if (service_ == nullptr) {
-    initService(sock);
-  }
-  CHECK(service_ != nullptr);
-
-  auto pipeline = RpcTestServerSerializePipeline::create();
-  pipeline->addBack(AsyncSocketHandler(sock));
-  // ensure we can write from any thread
-  pipeline->addBack(EventBaseHandler());
-  pipeline->addBack(LengthFieldBasedFrameDecoder());
-  pipeline->addBack(RpcTestServerSerializeHandler());
-  pipeline->addBack(MultiplexServerDispatcher<std::unique_ptr<Request>, std::unique_ptr<Response>>(
-      service_.get()));
-  pipeline->finalize();
-
-  return pipeline;
-}
-
-void RpcTestServerPipelineFactory::initService(std::shared_ptr<AsyncTransportWrapper> sock) {
-  /* get server address */
-  SocketAddress localAddress;
-  sock->getLocalAddress(&localAddress);
-
-  /* init service with server address */
-  service_ = std::make_shared<ExecutorFilter<std::unique_ptr<Request>, std::unique_ptr<Response>>>(
-      std::make_shared<CPUThreadPoolExecutor>(1),
-      std::make_shared<RpcTestService>(std::make_shared<SocketAddress>(localAddress)));
-}
-
-Future<std::unique_ptr<Response>> RpcTestService::operator()(std::unique_ptr<Request> request) {
-  /* build Response */
-  auto response = std::make_unique<Response>();
-  response->set_call_id(request->call_id());
-  std::string method_name = request->method();
-
-  if (method_name == "ping") {
-    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
-    response->set_resp_msg(pb_resp_msg);
-    VLOG(1) << "RPC server:"
-            << " ping called.";
-
-  } else if (method_name == "echo") {
-    auto pb_resp_msg = std::make_shared<EchoResponseProto>();
-    /* get msg from client */
-    auto pb_req_msg = std::static_pointer_cast<EchoRequestProto>(request->req_msg());
-    pb_resp_msg->set_message(pb_req_msg->message());
-    response->set_resp_msg(pb_resp_msg);
-    VLOG(1) << "RPC server:"
-            << " echo called, " << pb_req_msg->message();
-
-  } else if (method_name == "error") {
-    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
-    response->set_resp_msg(pb_resp_msg);
-    VLOG(1) << "RPC server:"
-            << " error called.";
-    response->set_exception(RpcTestException("server error!"));
-
-  } else if (method_name == "pause") {
-    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
-    /* sleeping */
-    auto pb_req_msg = std::static_pointer_cast<PauseRequestProto>(request->req_msg());
-    std::this_thread::sleep_for(std::chrono::milliseconds(pb_req_msg->ms()));
-    response->set_resp_msg(pb_resp_msg);
-    VLOG(1) << "RPC server:"
-            << " pause called, " << pb_req_msg->ms() << " ms";
-
-  } else if (method_name == "addr") {
-    // TODO:
-  } else if (method_name == "socketNotOpen") {
-    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
-    response->set_resp_msg(pb_resp_msg);
-  }
-
-  return folly::makeFuture<std::unique_ptr<Response>>(std::move(response));
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-test-server.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-test-server.h b/hbase-native-client/connection/rpc-test-server.h
deleted file mode 100644
index 955560e..0000000
--- a/hbase-native-client/connection/rpc-test-server.h
+++ /dev/null
@@ -1,68 +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.
- *
- */
-#pragma once
-#include <folly/SocketAddress.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/service/ExecutorFilter.h>
-#include <wangle/service/Service.h>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "exceptions/exception.h"
-
-using namespace hbase;
-using namespace folly;
-using namespace wangle;
-
-namespace hbase {
-using RpcTestServerSerializePipeline = wangle::Pipeline<IOBufQueue&, std::unique_ptr<Response>>;
-
-class RpcTestException : public IOException {
- public:
-  RpcTestException() {}
-  RpcTestException(const std::string& what) : IOException(what) {}
-  RpcTestException(const std::string& what, const folly::exception_wrapper& cause)
-      : IOException(what, cause) {}
-  RpcTestException(const folly::exception_wrapper& cause) : IOException("", cause) {}
-};
-
-class RpcTestService : public Service<std::unique_ptr<Request>, std::unique_ptr<Response>> {
- public:
-  RpcTestService(std::shared_ptr<folly::SocketAddress> socket_address)
-      : socket_address_(socket_address) {}
-  virtual ~RpcTestService() = default;
-  Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> request) override;
-
- private:
-  std::shared_ptr<folly::SocketAddress> socket_address_;
-};
-
-class RpcTestServerPipelineFactory : public PipelineFactory<RpcTestServerSerializePipeline> {
- public:
-  RpcTestServerSerializePipeline::Ptr newPipeline(
-      std::shared_ptr<AsyncTransportWrapper> sock) override;
-
- private:
-  void initService(std::shared_ptr<AsyncTransportWrapper> sock);
-
- private:
-  std::shared_ptr<ExecutorFilter<std::unique_ptr<Request>, std::unique_ptr<Response>>> service_{
-      nullptr};
-};
-}  // end of namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/rpc-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/rpc-test.cc b/hbase-native-client/connection/rpc-test.cc
deleted file mode 100644
index 8624e72..0000000
--- a/hbase-native-client/connection/rpc-test.cc
+++ /dev/null
@@ -1,284 +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 <wangle/bootstrap/ClientBootstrap.h>
-#include <wangle/channel/Handler.h>
-
-#include <folly/Format.h>
-#include <folly/Logging.h>
-#include <folly/SocketAddress.h>
-#include <folly/String.h>
-#include <folly/experimental/TestUtil.h>
-#include <folly/io/async/AsyncSocketException.h>
-#include <gflags/gflags.h>
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-#include <boost/thread.hpp>
-#include <chrono>
-
-#include "connection/rpc-client.h"
-#include "exceptions/exception.h"
-#include "if/test.pb.h"
-#include "rpc-test-server.h"
-#include "security/user.h"
-#include "serde/rpc-serde.h"
-
-using namespace wangle;
-using namespace folly;
-using namespace hbase;
-using namespace std::chrono;
-
-DEFINE_int32(port, 0, "test server port");
-DEFINE_string(result_format, "RPC {} returned: {}.", "output format of RPC result");
-DEFINE_string(fail_ex_format, "Shouldn't get here, exception is expected for RPC {}.",
-              "output format of enforcing fail with exception");
-DEFINE_string(fail_no_ex_format, "Shouldn't get here, exception is not expected for RPC {}.",
-              "output format of enforcing fail without exception");
-typedef ServerBootstrap<RpcTestServerSerializePipeline> ServerTestBootstrap;
-typedef std::shared_ptr<ServerTestBootstrap> ServerPtr;
-
-class RpcTest : public ::testing::Test {
- public:
-  static void SetUpTestCase() { google::InstallFailureSignalHandler(); }
-};
-
-std::shared_ptr<Configuration> CreateConf() {
-  auto conf = std::make_shared<Configuration>();
-  conf->Set(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE, "true");
-  return conf;
-}
-
-ServerPtr CreateRpcServer() {
-  /* create rpc test server */
-  auto server = std::make_shared<ServerTestBootstrap>();
-  server->childPipeline(std::make_shared<RpcTestServerPipelineFactory>());
-  server->bind(FLAGS_port);
-  return server;
-}
-
-std::shared_ptr<folly::SocketAddress> GetRpcServerAddress(ServerPtr server) {
-  auto addr = std::make_shared<folly::SocketAddress>();
-  server->getSockets()[0]->getAddress(addr.get());
-  return addr;
-}
-
-std::shared_ptr<RpcClient> CreateRpcClient(std::shared_ptr<Configuration> conf) {
-  auto io_executor = std::make_shared<wangle::IOThreadPoolExecutor>(1);
-  auto cpu_executor = std::make_shared<wangle::CPUThreadPoolExecutor>(1);
-  auto client = std::make_shared<RpcClient>(io_executor, cpu_executor, nullptr, conf);
-  return client;
-}
-
-std::shared_ptr<RpcClient> CreateRpcClient(std::shared_ptr<Configuration> conf,
-                                           std::chrono::nanoseconds connect_timeout) {
-  auto io_executor = std::make_shared<wangle::IOThreadPoolExecutor>(1);
-  auto cpu_executor = std::make_shared<wangle::CPUThreadPoolExecutor>(1);
-  auto client =
-      std::make_shared<RpcClient>(io_executor, cpu_executor, nullptr, conf, connect_timeout);
-  return client;
-}
-
-/**
-* test ping
-*/
-TEST_F(RpcTest, Ping) {
-  auto conf = CreateConf();
-  auto server = CreateRpcServer();
-  auto server_addr = GetRpcServerAddress(server);
-  auto client = CreateRpcClient(conf);
-
-  auto method = "ping";
-  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                           std::make_shared<EmptyResponseProto>(), method);
-
-  /* sending out request */
-  client
-      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
-                  hbase::security::User::defaultUser())
-      .then([&](std::unique_ptr<Response> response) {
-        auto pb_resp = std::static_pointer_cast<EmptyResponseProto>(response->resp_msg());
-        EXPECT_TRUE(pb_resp != nullptr);
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, "");
-      })
-      .onError([&](const folly::exception_wrapper& ew) {
-        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
-      })
-      .get();
-
-  server->stop();
-  server->join();
-}
-
-/**
- * test echo
- */
-TEST_F(RpcTest, Echo) {
-  auto conf = CreateConf();
-  auto server = CreateRpcServer();
-  auto server_addr = GetRpcServerAddress(server);
-  auto client = CreateRpcClient(conf);
-
-  auto method = "echo";
-  auto greetings = "hello, hbase server!";
-  auto request = std::make_unique<Request>(std::make_shared<EchoRequestProto>(),
-                                           std::make_shared<EchoResponseProto>(), method);
-  auto pb_msg = std::static_pointer_cast<EchoRequestProto>(request->req_msg());
-  pb_msg->set_message(greetings);
-
-  /* sending out request */
-  client
-      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
-                  hbase::security::User::defaultUser())
-      .then([&](std::unique_ptr<Response> response) {
-        auto pb_resp = std::static_pointer_cast<EchoResponseProto>(response->resp_msg());
-        EXPECT_TRUE(pb_resp != nullptr);
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, pb_resp->message());
-        EXPECT_EQ(greetings, pb_resp->message());
-      })
-      .onError([&](const folly::exception_wrapper& ew) {
-        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
-      })
-      .get();
-
-  server->stop();
-  server->join();
-}
-
-/**
- * test error
- */
-TEST_F(RpcTest, Error) {
-  auto conf = CreateConf();
-  auto server = CreateRpcServer();
-  auto server_addr = GetRpcServerAddress(server);
-  auto client = CreateRpcClient(conf);
-
-  auto method = "error";
-  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                           std::make_shared<EmptyResponseProto>(), method);
-  /* sending out request */
-  client
-      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
-                  hbase::security::User::defaultUser())
-      .then([&](std::unique_ptr<Response> response) {
-        FAIL() << folly::sformat(FLAGS_fail_ex_format, method);
-      })
-      .onError([&](const folly::exception_wrapper& ew) {
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
-        std::string kRemoteException = demangle(typeid(hbase::RemoteException)).toStdString();
-        std::string kRpcTestException = demangle(typeid(hbase::RpcTestException)).toStdString();
-
-        /* verify exception_wrapper */
-        EXPECT_TRUE(bool(ew));
-        EXPECT_EQ(kRemoteException, ew.class_name());
-
-        /* verify exception */
-        EXPECT_TRUE(ew.with_exception([&](const hbase::RemoteException& e) {
-          EXPECT_EQ(kRpcTestException, e.exception_class_name());
-          EXPECT_EQ(kRpcTestException + ": server error!", e.stack_trace());
-        }));
-      })
-      .get();
-
-  server->stop();
-  server->join();
-}
-
-TEST_F(RpcTest, SocketNotOpen) {
-  auto conf = CreateConf();
-  auto server = CreateRpcServer();
-  auto server_addr = GetRpcServerAddress(server);
-  auto client = CreateRpcClient(conf);
-
-  auto method = "socketNotOpen";
-  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
-                                           std::make_shared<EmptyResponseProto>(), method);
-
-  server->stop();
-  server->join();
-
-  /* sending out request */
-  client
-      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
-                  hbase::security::User::defaultUser())
-      .then([&](std::unique_ptr<Response> response) {
-        FAIL() << folly::sformat(FLAGS_fail_ex_format, method);
-      })
-      .onError([&](const folly::exception_wrapper& ew) {
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
-        std::string kConnectionException =
-            demangle(typeid(hbase::ConnectionException)).toStdString();
-        std::string kAsyncSocketException =
-            demangle(typeid(folly::AsyncSocketException)).toStdString();
-
-        /* verify exception_wrapper */
-        EXPECT_TRUE(bool(ew));
-        EXPECT_EQ(kConnectionException, ew.class_name());
-
-        /* verify exception */
-        EXPECT_TRUE(ew.with_exception([&](const hbase::ConnectionException& e) {
-          EXPECT_TRUE(bool(e.cause()));
-          EXPECT_EQ(kAsyncSocketException, e.cause().class_name());
-          VLOG(1) << folly::sformat(FLAGS_result_format, method, e.cause().what());
-          e.cause().with_exception([&](const folly::AsyncSocketException& ase) {
-            EXPECT_EQ(AsyncSocketException::AsyncSocketExceptionType::NOT_OPEN, ase.getType());
-            EXPECT_EQ(111 /*ECONNREFUSED*/, ase.getErrno());
-          });
-        }));
-      })
-      .get();
-}
-
-/**
- * test pause
- */
-TEST_F(RpcTest, Pause) {
-  int ms = 500;
-
-  auto conf = CreateConf();
-  auto server = CreateRpcServer();
-  auto server_addr = GetRpcServerAddress(server);
-  auto client =
-      CreateRpcClient(conf, std::chrono::duration_cast<nanoseconds>(milliseconds(2 * ms)));
-
-  auto method = "pause";
-  auto request = std::make_unique<Request>(std::make_shared<PauseRequestProto>(),
-                                           std::make_shared<EmptyResponseProto>(), method);
-  auto pb_msg = std::static_pointer_cast<PauseRequestProto>(request->req_msg());
-
-  pb_msg->set_ms(ms);
-
-  /* sending out request */
-  client
-      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
-                  hbase::security::User::defaultUser())
-      .then([&](std::unique_ptr<Response> response) {
-        auto pb_resp = std::static_pointer_cast<EmptyResponseProto>(response->resp_msg());
-        EXPECT_TRUE(pb_resp != nullptr);
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, "");
-      })
-      .onError([&](const folly::exception_wrapper& ew) {
-        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
-        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
-      })
-      .get();
-
-  server->stop();
-  server->join();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/sasl-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/sasl-handler.cc b/hbase-native-client/connection/sasl-handler.cc
deleted file mode 100644
index 9afe1e2..0000000
--- a/hbase-native-client/connection/sasl-handler.cc
+++ /dev/null
@@ -1,225 +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 "connection/sasl-handler.h"
-
-#include <glog/logging.h>
-#include <sasl/sasl.h>
-#include <sasl/saslplug.h>
-#include <sasl/saslutil.h>
-
-#include <google/protobuf/io/coded_stream.h>
-#include <google/protobuf/io/zero_copy_stream_impl.h>
-#include <wangle/channel/Handler.h>
-
-#include <condition_variable>
-#include <memory>
-#include <mutex>
-#include <string>
-#include <utility>
-
-#include "connection/service.h"
-#include "security/user.h"
-using hbase::security::User;
-
-using std::chrono::nanoseconds;
-using namespace folly;
-using namespace wangle;
-using namespace hbase;
-
-SaslHandler::SaslHandler(std::string user_name, std::shared_ptr<Configuration> conf)
-    : user_name_(user_name) {
-  host_name_.clear();
-  secure_ = User::IsSecurityEnabled(*conf);
-  service_name_ = SaslUtil::ParseServiceName(conf, secure_);
-  sasl_connection_setup_started_.clear();
-  sasl_connection_setup_in_progress_.store(true);
-}
-
-SaslHandler::SaslHandler(const SaslHandler &hdlr) {
-  user_name_ = hdlr.user_name_;
-  service_name_ = hdlr.service_name_;
-  secure_ = hdlr.secure_;
-  host_name_ = hdlr.host_name_;
-  // copy-constructor sets the flags below to their initial state as opposed to getting them
-  // from the object this class is constructed from. That way, this instance is ready to do
-  // sasl stuff without issues, right from the SaslInit. Sharing a sasl session is not useful
-  // between two handler instances.
-  sasl_connection_setup_started_.clear();
-  sasl_connection_setup_in_progress_.store(true);
-  sconn_ = nullptr;
-}
-
-SaslHandler::~SaslHandler() {
-  if (nullptr != sconn_) {
-    sasl_dispose(&sconn_);
-  }
-  sconn_ = nullptr;
-}
-
-void SaslHandler::transportActive(Context *ctx) {
-  // assign hostname; needed for the sasl handshake if secure
-  folly::SocketAddress address;
-  ctx->getTransport()->getPeerAddress(&address);
-  host_name_ = address.getHostStr();
-
-  // now init the sasl library; this is once per process
-  if (secure_) {
-    sasl_util_.InitializeSaslLib();
-  }
-  // write the preamble to kick off the RPC handshake
-  VLOG(3) << "Writing RPC connection Preamble to server: " << host_name_;
-  auto preamble = RpcSerde::Preamble(secure_);
-  ctx->fireWrite(std::move(preamble));
-  ctx->fireTransportActive();
-}
-
-void SaslHandler::read(Context *ctx, folly::IOBufQueue &buf) {
-  // if security is not on, or in case of security-on, if secure connection setup not in progress,
-  // pass it up without touching
-  if (!secure_ || !sasl_connection_setup_in_progress_.load()) {
-    ctx->fireRead(buf);
-  } else {
-    // message is for this handler; process it appropriately
-    ContinueSaslNegotiation(ctx, &buf);
-  }
-}
-
-folly::Future<folly::Unit> SaslHandler::write(Context *ctx, std::unique_ptr<folly::IOBuf> buf) {
-  // if security is on, and if secure connection setup in progress,
-  // this message is for this handler to process and respond
-  if (secure_ && sasl_connection_setup_in_progress_.load()) {
-    // store IOBuf which is to be sent to server after SASL handshake
-    iobuf_.push_back(std::move(buf));
-    if (!sasl_connection_setup_started_.test_and_set()) {
-      // for the first incoming RPC from the higher layer, trigger sasl initialization
-      return SaslInit(ctx);
-    } else {
-      // for the subsequent incoming RPCs from the higher layer, just return empty future
-      folly::Promise<folly::Unit> p_;
-      return p_.getFuture();
-    }
-  }
-  // pass the bytes recieved down without touching it
-  return ctx->fireWrite(std::move(buf));
-}
-
-folly::Future<folly::Unit> SaslHandler::WriteSaslOutput(Context *ctx, const char *out,
-                                                        unsigned int outlen) {
-  int buffer_size = outlen + 4;
-  auto iob = IOBuf::create(buffer_size);
-  iob->append(buffer_size);
-  // Create the array output stream.
-  google::protobuf::io::ArrayOutputStream aos{iob->writableData(), buffer_size};
-  std::unique_ptr<google::protobuf::io::CodedOutputStream> coded_output =
-      std::make_unique<google::protobuf::io::CodedOutputStream>(&aos);
-  uint32_t total_size = outlen;
-  total_size = ntohl(total_size);
-  coded_output->WriteRaw(&total_size, 4);
-  coded_output->WriteRaw(out, outlen);
-  return ctx->fireWrite(std::move(iob));
-}
-
-void SaslHandler::FinishAuth(Context *ctx, folly::IOBufQueue *bufQueue) {
-  std::unique_ptr<folly::IOBuf> iob;
-  if (!bufQueue->empty()) {
-    iob = bufQueue->pop_front();
-    throw std::runtime_error("Error in the final step of handshake " +
-                             std::string(reinterpret_cast<const char *>(iob->data())));
-  } else {
-    sasl_connection_setup_in_progress_.store(false);
-    // write what we buffered
-    for (size_t i = 0; i < iobuf_.size(); i++) {
-      iob = std::move(iobuf_.at(i));
-      ctx->fireWrite(std::move(iob));
-    }
-  }
-}
-
-folly::Future<folly::Unit> SaslHandler::SaslInit(Context *ctx) {
-  int rc;
-  const char *mechusing, *mechlist = "GSSAPI";
-  const char *out;
-  unsigned int outlen;
-
-  rc = sasl_client_new(service_name_.c_str(), /* The service we are using*/
-                       host_name_.c_str(), NULL,
-                       NULL, /* Local and remote IP address strings
-                                   (NULL disables mechanisms which require this info)*/
-                       NULL, /*connection-specific callbacks*/
-                       0 /*security flags*/, &sconn_);
-  if (rc != SASL_OK) {
-    LOG(FATAL) << "Cannot create client (" << rc << ") ";
-    throw std::runtime_error("Cannot create client");
-  }
-  int curr_rc;
-  do {
-    curr_rc = sasl_client_start(sconn_,   /* the same context from above */
-                                mechlist, /* the list of mechanisms from the server */
-                                NULL,     /* filled in if an interaction is needed */
-                                &out,     /* filled in on success */
-                                &outlen,  /* filled in on success */
-                                &mechusing);
-  } while (curr_rc == SASL_INTERACT); /* the mechanism may ask us to fill
-     in things many times. result is SASL_CONTINUE on success */
-  if (curr_rc != SASL_CONTINUE) {
-    throw std::runtime_error("Cannot start client (" + std::to_string(curr_rc) + ")");
-  }
-  folly::Future<folly::Unit> fut = WriteSaslOutput(ctx, out, outlen);
-  return fut;
-}
-
-void SaslHandler::ContinueSaslNegotiation(Context *ctx, folly::IOBufQueue *bufQueue) {
-  const char *out;
-  unsigned int outlen;
-
-  int bytes_sent = 0;
-  int bytes_received = 0;
-
-  std::unique_ptr<folly::IOBuf> iob = bufQueue->pop_front();
-  bytes_received = iob->length();
-  if (bytes_received == 0) {
-    throw std::runtime_error("Error in sasl handshake");
-  }
-  folly::io::RWPrivateCursor c(iob.get());
-  std::uint32_t status = c.readBE<std::uint32_t>();
-  std::uint32_t sz = c.readBE<std::uint32_t>();
-
-  if (status != 0 /*Status 0 is success*/) {
-    // Assumption here is that the response from server is not more than 8 * 1024
-    throw std::runtime_error("Error in sasl handshake " +
-                             std::string(reinterpret_cast<char *>(c.writableData())));
-  }
-  out = nullptr;
-  outlen = 0;
-
-  int curr_rc =
-      sasl_client_step(sconn_,                                     /* our context */
-                       reinterpret_cast<char *>(c.writableData()), /* the data from the server */
-                       sz,                                         /* its length */
-                       NULL,     /* this should be unallocated and NULL */
-                       &out,     /* filled in on success */
-                       &outlen); /* filled in on success */
-
-  if (curr_rc == SASL_OK || curr_rc == SASL_CONTINUE) {
-    WriteSaslOutput(ctx, out, outlen);
-  }
-  if (curr_rc == SASL_OK) {
-    FinishAuth(ctx, bufQueue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/sasl-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/sasl-handler.h b/hbase-native-client/connection/sasl-handler.h
deleted file mode 100644
index 81f4e81..0000000
--- a/hbase-native-client/connection/sasl-handler.h
+++ /dev/null
@@ -1,78 +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.
- *
- */
-#pragma once
-
-#include <glog/logging.h>
-#include <sasl/sasl.h>
-#include <sasl/saslplug.h>
-#include <sasl/saslutil.h>
-
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "connection/sasl-util.h"
-#include "connection/service.h"
-#include "security/user.h"
-#include "serde/rpc-serde.h"
-
-namespace hbase {
-
-/**
- * Class to perform SASL handshake with server (currently works with regionserver principals only)
- * It is inserted between EventBaseHandler and LengthFieldBasedFrameDecoder in the pipeline
- * SaslHandler would intercept writes to server by buffering the IOBuf's and start the handshake
- * process
- *   (via sasl_client_XX calls provided by Cyrus)
- * After handshake is complete, SaslHandler would send the buffered IOBuf's to server and
- *   act as pass-thru from then on
- */
-class SaslHandler
-    : public wangle::HandlerAdapter<folly::IOBufQueue&, std::unique_ptr<folly::IOBuf>> {
- public:
-  explicit SaslHandler(std::string user_name, std::shared_ptr<Configuration> conf);
-  SaslHandler(const SaslHandler& hdlr);
-  ~SaslHandler();
-
-  // from HandlerAdapter
-  void read(Context* ctx, folly::IOBufQueue& buf) override;
-  folly::Future<folly::Unit> write(Context* ctx, std::unique_ptr<folly::IOBuf> buf) override;
-  void transportActive(Context* ctx) override;
-
- private:
-  // used by Cyrus
-  sasl_conn_t* sconn_ = nullptr;
-  std::string user_name_;
-  std::string service_name_;
-  std::string host_name_;
-  bool secure_;
-  std::atomic_flag sasl_connection_setup_started_;
-  std::atomic<bool> sasl_connection_setup_in_progress_{true};
-  // vector of folly::IOBuf which buffers client writes before handshake is complete
-  std::vector<std::unique_ptr<folly::IOBuf>> iobuf_;
-  SaslUtil sasl_util_;
-
-  // writes the output returned by sasl_client_XX to server
-  folly::Future<folly::Unit> WriteSaslOutput(Context* ctx, const char* out, unsigned int outlen);
-  folly::Future<folly::Unit> SaslInit(Context* ctx);
-  void FinishAuth(Context* ctx, folly::IOBufQueue* bufQueue);
-  void ContinueSaslNegotiation(Context* ctx, folly::IOBufQueue* buf);
-  std::string ParseServiceName(std::shared_ptr<Configuration> conf, bool secure);
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/sasl-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/sasl-util.cc b/hbase-native-client/connection/sasl-util.cc
deleted file mode 100644
index ecaf015..0000000
--- a/hbase-native-client/connection/sasl-util.cc
+++ /dev/null
@@ -1,92 +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 "connection/sasl-util.h"
-
-#include <glog/logging.h>
-#include <sasl/sasl.h>
-#include <sasl/saslplug.h>
-#include <sasl/saslutil.h>
-
-#include <string>
-
-int SaslUtil::GetPluginPath(void *context __attribute__((unused)), const char **path) {
-  *path = getenv("SASL_PATH");
-
-  if (*path == NULL) {
-    *path = kDefaultPluginDir;
-  }
-  return SASL_OK;
-}
-
-void *SaslUtil::MutexNew(void) {
-  auto m = new std::mutex();
-  return m;
-}
-
-int SaslUtil::MutexLock(void *m) {
-  (reinterpret_cast<std::mutex *>(m))->lock();
-  return SASL_OK;
-}
-
-int SaslUtil::MutexUnlock(void *m) {
-  (reinterpret_cast<std::mutex *>(m))->unlock();
-  return SASL_OK;
-}
-
-void SaslUtil::MutexDispose(void *m) {
-  std::mutex *mutex = reinterpret_cast<std::mutex *>(m);
-  delete mutex;
-}
-
-std::once_flag SaslUtil::library_inited_;
-
-void SaslUtil::InitializeSaslLib() {
-  std::call_once(library_inited_, []() {
-    sasl_set_mutex(reinterpret_cast<sasl_mutex_alloc_t *>(&SaslUtil::MutexNew),
-                   reinterpret_cast<sasl_mutex_lock_t *>(&SaslUtil::MutexLock),
-                   reinterpret_cast<sasl_mutex_unlock_t *>(&SaslUtil::MutexUnlock),
-                   reinterpret_cast<sasl_mutex_free_t *>(&SaslUtil::MutexDispose));
-    static sasl_callback_t callbacks[] = {
-        {SASL_CB_GETPATH, (sasl_callback_ft)&SaslUtil::GetPluginPath, NULL},
-        {SASL_CB_LIST_END, NULL, NULL}};
-    int rc = sasl_client_init(callbacks);
-    if (rc != SASL_OK) {
-      throw std::runtime_error("Cannot initialize client " + std::to_string(rc));
-    }
-  });
-}
-
-std::string SaslUtil::ParseServiceName(std::shared_ptr<hbase::Configuration> conf, bool secure) {
-  if (!secure) {
-    return std::string();
-  }
-  std::string svrPrincipal = conf->Get(kServerPrincipalConfKey, "");
-  // principal is of this form: hbase/23a03935850c@EXAMPLE.COM
-  // where 23a03935850c is the host (optional)
-  std::size_t pos = svrPrincipal.find("/");
-  if (pos == std::string::npos && svrPrincipal.find("@") != std::string::npos) {
-    pos = svrPrincipal.find("@");
-  }
-  if (pos == std::string::npos) {
-    throw std::runtime_error("Couldn't retrieve service principal from conf");
-  }
-  VLOG(1) << "pos " << pos << " " << svrPrincipal;
-  std::string service_name = svrPrincipal.substr(0, pos);
-  return service_name;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/sasl-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/sasl-util.h b/hbase-native-client/connection/sasl-util.h
deleted file mode 100644
index 4d58d9ee..0000000
--- a/hbase-native-client/connection/sasl-util.h
+++ /dev/null
@@ -1,46 +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.
- *
- */
-#pragma once
-
-#include <memory>
-#include <mutex>
-#include <string>
-
-#include "core/configuration.h"
-
-class SaslUtil {
- public:
-  void InitializeSaslLib(void);
-  static std::string ParseServiceName(std::shared_ptr<hbase::Configuration> conf, bool secure);
-
- private:
-  static constexpr const char *kDefaultPluginDir = "/usr/lib/sasl2";
-  // for now the sasl handler is hardcoded to work against the regionservers only. In the future, if
-  // we
-  // need the master rpc to work, we could have a map of service names to principals to use (similar
-  // to the Java implementation)
-  static constexpr const char *kServerPrincipalConfKey = "hbase.regionserver.kerberos.principal";
-
-  static int GetPluginPath(void *context, const char **path);
-  static void *MutexNew(void);
-  static int MutexLock(void *m);
-  static int MutexUnlock(void *m);
-  static void MutexDispose(void *m);
-  static std::once_flag library_inited_;
-};

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/service.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/service.h b/hbase-native-client/connection/service.h
deleted file mode 100644
index 64d4f07..0000000
--- a/hbase-native-client/connection/service.h
+++ /dev/null
@@ -1,30 +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.
- *
- */
-#pragma once
-
-#include <wangle/service/Service.h>
-
-#include <memory>
-
-#include "connection/request.h"
-#include "connection/response.h"
-
-namespace hbase {
-using HBaseService = wangle::Service<std::unique_ptr<Request>, std::unique_ptr<Response>>;
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
deleted file mode 100644
index 76c836b..0000000
--- a/hbase-native-client/core/BUCK
+++ /dev/null
@@ -1,348 +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.
-
-# This is the main library.
-cxx_library(
-    name="core",
-    exported_headers=[
-        "async-client-scanner.h",
-        "async-connection.h",
-        "async-region-locator.h",
-        "async-rpc-retrying-caller-factory.h",
-        "async-rpc-retrying-caller.h",
-        "async-table-result-scanner.h",
-        "client.h",
-        "cell.h",
-        "hbase-macros.h",
-        "filter.h",
-        "query.h",
-        "keyvalue-codec.h",
-        "region-location.h",
-        "location-cache.h",
-        "connection-configuration.h",
-        # TODO: move this out of exported
-        # Once meta lookup works
-        "meta-utils.h",
-        "get.h",
-        "increment.h",
-        "mutation.h",
-        "put.h",
-        "delete.h",
-        "scan.h",
-        "append.h",
-        "result.h",
-        "result-scanner.h",
-        "request-converter.h",
-        "response-converter.h",
-        "table.h",
-        "async-scan-rpc-retrying-caller.h",
-        "raw-async-table.h",
-        "raw-scan-result-consumer.h",
-        "scan-result-cache.h",
-        "hbase-rpc-controller.h",
-        "time-range.h",
-        "zk-util.h",
-        "action.h",
-        "multi-response.h",
-        "region-request.h",
-        "region-result.h",
-        "row.h",
-        "server-request.h",
-        "async-batch-rpc-retrying-caller.h",
-    ],
-    srcs=[
-        "async-client-scanner.cc",
-        "async-connection.cc",
-        "async-rpc-retrying-caller-factory.cc",
-        "async-rpc-retrying-caller.cc",
-        "async-scan-rpc-retrying-caller.cc",
-        "async-table-result-scanner.cc",
-        "cell.cc",
-        "client.cc",
-        "hbase-rpc-controller.cc",
-        "keyvalue-codec.cc",
-        "location-cache.cc",
-        "meta-utils.cc",
-        "increment.cc",
-        "get.cc",
-        "mutation.cc",
-        "put.cc",
-        "delete.cc",
-        "scan.cc",
-        "append.cc",
-        "scan-result-cache.cc",
-        "raw-async-table.cc",
-        "result.cc",
-        "request-converter.cc",
-        "response-converter.cc",
-        "table.cc",
-        "time-range.cc",
-        "zk-util.cc",
-        "multi-response.cc",
-        "region-result.cc",
-        "async-batch-rpc-retrying-caller.cc",
-    ],
-    deps=[
-        "//exceptions:exceptions",
-        "//utils:utils",
-        "//connection:connection",
-        "//core:conf",
-        "//if:if",
-        "//serde:serde",
-        "//third-party:folly",
-        "//third-party:wangle",
-        "//third-party:zookeeper_mt",
-    ],
-    compiler_flags=['-Weffc++', '-ggdb'],
-    visibility=[
-        'PUBLIC',
-    ],)
-cxx_library(
-    name="conf",
-    exported_headers=[
-        "configuration.h",
-        "hbase-configuration-loader.h",
-    ],
-    srcs=[
-        "configuration.cc",
-        "hbase-configuration-loader.cc",
-    ],
-    deps=["//utils:utils", "//third-party:folly"],
-    compiler_flags=['-Weffc++', '-ggdb'],
-    visibility=[
-        'PUBLIC',
-    ],)
-cxx_test(
-    name="location-cache-test",
-    srcs=[
-        "location-cache-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//test-util:test-util",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="location-cache-retry-test",
-    srcs=[
-        "location-cache-retry-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//if:if",
-        "//serde:serde",
-        "//test-util:test-util",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="cell-test",
-    srcs=[
-        "cell-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="filter-test",
-    srcs=[
-        "filter-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//if:if",
-        "//serde:serde",
-        "//test-util:test-util",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="get-test",
-    srcs=[
-        "get-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="delete-test",
-    srcs=[
-        "delete-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="increment-test",
-    srcs=[
-        "increment-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="put-test",
-    srcs=[
-        "put-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="append-test",
-    srcs=[
-        "append-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="retry-test",
-    srcs=[
-        "async-rpc-retrying-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//test-util:test-util",
-        "//exceptions:exceptions",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="time-range-test",
-    srcs=[
-        "time-range-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="configuration-test",
-    srcs=[
-        "configuration-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="hbase-configuration-test",
-    srcs=[
-        "hbase-configuration-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="scan-test",
-    srcs=[
-        "scan-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="result-test",
-    srcs=[
-        "result-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="request-converter-test",
-    srcs=[
-        "request-converter-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//connection:connection",
-        "//if:if",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="client-test",
-    srcs=[
-        "client-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//if:if",
-        "//serde:serde",
-        "//test-util:test-util",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="scan-result-cache-test",
-    srcs=[
-        "scan-result-cache-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="scanner-test",
-    srcs=[
-        "scanner-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//if:if",
-        "//serde:serde",
-        "//test-util:test-util",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="zk-util-test",
-    srcs=[
-        "zk-util-test.cc",
-    ],
-    deps=[
-        ":core",
-    ],
-    run_test_separately=True,)
-cxx_test(
-    name="multi-retry-test",
-    srcs=[
-        "async-batch-rpc-retrying-test.cc",
-    ],
-    deps=[
-        ":core",
-        "//test-util:test-util",
-        "//exceptions:exceptions",
-    ],
-    run_test_separately=True,)
-cxx_binary(
-    name="simple-client",
-    srcs=[
-        "simple-client.cc",
-    ],
-    deps=[":core", "//connection:connection"],)
-cxx_binary(
-    name="load-client",
-    srcs=[
-        "load-client.cc",
-    ],
-    deps=[":core", "//connection:connection"],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/action.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/action.h b/hbase-native-client/core/action.h
deleted file mode 100644
index a00f079..0000000
--- a/hbase-native-client/core/action.h
+++ /dev/null
@@ -1,43 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include "core/row.h"
-
-namespace hbase {
-class Action {
- public:
-  Action(std::shared_ptr<hbase::Row> action, int32_t original_index)
-      : action_(action), original_index_(original_index) {}
-  ~Action() {}
-
-  int32_t original_index() const { return original_index_; }
-
-  std::shared_ptr<hbase::Row> action() const { return action_; }
-
- private:
-  std::shared_ptr<hbase::Row> action_;
-  int32_t original_index_;
-  int64_t nonce_ = -1;
-  int32_t replica_id_ = -1;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/append-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/append-test.cc b/hbase-native-client/core/append-test.cc
deleted file mode 100644
index 2216034..0000000
--- a/hbase-native-client/core/append-test.cc
+++ /dev/null
@@ -1,105 +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 <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include "core/append.h"
-#include "core/mutation.h"
-#include "utils/time-util.h"
-
-using hbase::Append;
-using hbase::Cell;
-using hbase::CellType;
-using hbase::Mutation;
-using hbase::TimeUtil;
-
-const constexpr int64_t Mutation::kLatestTimestamp;
-
-TEST(Append, Row) {
-  Append append{"foo"};
-  EXPECT_EQ("foo", append.row());
-}
-
-TEST(Append, Durability) {
-  Append append{"row"};
-  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, append.Durability());
-
-  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
-  append.SetDurability(skipWal);
-  EXPECT_EQ(skipWal, append.Durability());
-}
-
-TEST(Append, Timestamp) {
-  Append append{"row"};
-
-  // test default timestamp
-  EXPECT_EQ(Mutation::kLatestTimestamp, append.TimeStamp());
-
-  // set custom timestamp
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  append.SetTimeStamp(ts);
-  EXPECT_EQ(ts, append.TimeStamp());
-
-  // Add a column with custom timestamp
-  append.Add("f", "q", "v");
-  auto &cell = append.FamilyMap().at("f")[0];
-  EXPECT_EQ(ts, cell->Timestamp());
-}
-
-TEST(Append, HasFamilies) {
-  Append append{"row"};
-
-  EXPECT_EQ(false, append.HasFamilies());
-
-  append.Add("f", "q", "v");
-  EXPECT_EQ(true, append.HasFamilies());
-}
-
-TEST(Append, Add) {
-  CellType cell_type = CellType::PUT;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-
-  // add first cell
-  Append append{"row"};
-  append.Add(std::move(cell));
-  EXPECT_EQ(1, append.FamilyMap().size());
-  EXPECT_EQ(1, append.FamilyMap().at(family).size());
-
-  // add a non-matching row
-  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-  Append append2{"foo"};
-  ASSERT_THROW(append2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
-
-  // add a second cell with same family
-  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
-  append.Add(std::move(cell3));
-  EXPECT_EQ(1, append.FamilyMap().size());
-  EXPECT_EQ(2, append.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
-  append.Add(std::move(cell4));
-  EXPECT_EQ(2, append.FamilyMap().size());
-  EXPECT_EQ(1, append.FamilyMap().at("family-2").size());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/append.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/append.cc b/hbase-native-client/core/append.cc
deleted file mode 100644
index 95349ae..0000000
--- a/hbase-native-client/core/append.cc
+++ /dev/null
@@ -1,53 +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/append.h"
-#include <folly/Conv.h>
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-#include <utility>
-
-namespace hbase {
-
-/**
- *  @brief Append to the column from the specific family with the specified qualifier
- *  @param family family name
- *  @param qualifier column qualifier
- *  @param value value to append
- */
-Append& Append::Add(const std::string& family, const std::string& qualifier,
-                    const std::string& value) {
-  family_map_[family].push_back(std::move(
-      std::make_unique<Cell>(row_, family, qualifier, timestamp_, value, hbase::CellType::PUT)));
-  return *this;
-}
-Append& Append::Add(std::unique_ptr<Cell> cell) {
-  if (cell->Row() != row_) {
-    throw std::runtime_error("The row in " + cell->DebugString() +
-                             " doesn't match the original one " + row_);
-  }
-
-  family_map_[cell->Family()].push_back(std::move(cell));
-  return *this;
-}
-
-}  // namespace hbase


[18/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/raw-async-table.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/raw-async-table.cc b/hbase-native-client/core/raw-async-table.cc
deleted file mode 100644
index 409883f..0000000
--- a/hbase-native-client/core/raw-async-table.cc
+++ /dev/null
@@ -1,259 +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 <utility>
-
-#include "core/raw-async-table.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-
-using hbase::security::User;
-
-namespace hbase {
-
-template <typename RESP>
-std::shared_ptr<SingleRequestCallerBuilder<RESP>> RawAsyncTable::CreateCallerBuilder(
-    std::string row, std::chrono::nanoseconds rpc_timeout) {
-  return connection_->caller_factory()
-      ->Single<RESP>()
-      ->table(table_name_)
-      ->row(row)
-      ->rpc_timeout(rpc_timeout)
-      ->operation_timeout(connection_conf_->operation_timeout())
-      ->pause(connection_conf_->pause())
-      ->max_retries(connection_conf_->max_retries())
-      ->start_log_errors_count(connection_conf_->start_log_errors_count());
-}
-
-template <typename REQ, typename PREQ, typename PRESP, typename RESP>
-folly::Future<RESP> RawAsyncTable::Call(
-    std::shared_ptr<RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
-    std::shared_ptr<RegionLocation> loc, const REQ& req,
-    const ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
-    const RespConverter<RESP, PRESP> resp_converter) {
-  std::unique_ptr<PREQ> preq = req_converter(req, loc->region_name());
-
-  // No need to make take a callable argument, it is always the same
-  return rpc_client
-      ->AsyncCall(loc->server_name().host_name(), loc->server_name().port(), std::move(preq),
-                  User::defaultUser(), "ClientService")
-      .then(
-          [resp_converter](const std::unique_ptr<PRESP>& presp) { return resp_converter(*presp); });
-}
-
-folly::Future<std::shared_ptr<Result>> RawAsyncTable::Get(const hbase::Get& get) {
-  auto caller =
-      CreateCallerBuilder<std::shared_ptr<Result>>(get.row(), connection_conf_->read_rpc_timeout())
-          ->action([=, &get](std::shared_ptr<hbase::HBaseRpcController> controller,
-                             std::shared_ptr<hbase::RegionLocation> loc,
-                             std::shared_ptr<hbase::RpcClient> rpc_client)
-                       -> folly::Future<std::shared_ptr<hbase::Result>> {
-                         return Call<hbase::Get, hbase::Request, hbase::Response,
-                                     std::shared_ptr<hbase::Result>>(
-                             rpc_client, controller, loc, get,
-                             &hbase::RequestConverter::ToGetRequest,
-                             &hbase::ResponseConverter::FromGetResponse);
-                       })
-          ->Build();
-
-  // Return the Future we obtain from the call(). However, we do not want the Caller to go out of
-  // context and get deallocated since the caller injects a lot of closures which capture [this, &]
-  // which is use-after-free. We are just passing an identity closure capturing caller by value to
-  // ensure  that the lifecycle of the Caller object is longer than the retry lambdas.
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-folly::Future<std::shared_ptr<Result>> RawAsyncTable::Increment(const hbase::Increment& incr) {
-  auto caller =
-      CreateCallerBuilder<std::shared_ptr<Result>>(incr.row(),
-                                                   connection_conf_->write_rpc_timeout())
-          ->action([=, &incr](std::shared_ptr<hbase::HBaseRpcController> controller,
-                              std::shared_ptr<hbase::RegionLocation> loc,
-                              std::shared_ptr<hbase::RpcClient>
-                                  rpc_client) -> folly::Future<std::shared_ptr<Result>> {
-            return Call<hbase::Increment, hbase::Request, hbase::Response, std::shared_ptr<Result>>(
-                rpc_client, controller, loc, incr,
-                &hbase::RequestConverter::IncrementToMutateRequest,
-                &hbase::ResponseConverter::FromMutateResponse);
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<folly::Unit> RawAsyncTable::Put(const hbase::Put& put) {
-  auto caller =
-      CreateCallerBuilder<folly::Unit>(put.row(), connection_conf_->write_rpc_timeout())
-          ->action([=, &put](
-                       std::shared_ptr<hbase::HBaseRpcController> controller,
-                       std::shared_ptr<hbase::RegionLocation> loc,
-                       std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<folly::Unit> {
-            return Call<hbase::Put, hbase::Request, hbase::Response, folly::Unit>(
-                rpc_client, controller, loc, put, &hbase::RequestConverter::ToMutateRequest,
-                [](const Response& r) -> folly::Unit { return folly::unit; });
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<bool> RawAsyncTable::CheckAndPut(const std::string& row, const std::string& family,
-                                               const std::string& qualifier,
-                                               const std::string& value, const hbase::Put& put,
-                                               const pb::CompareType& compare_op) {
-  auto caller =
-      CreateCallerBuilder<bool>(row, connection_conf_->write_rpc_timeout())
-          ->action([=, &put](std::shared_ptr<hbase::HBaseRpcController> controller,
-                             std::shared_ptr<hbase::RegionLocation> loc,
-                             std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<bool> {
-            return Call<hbase::Put, hbase::Request, hbase::Response, bool>(
-                rpc_client, controller, loc, put,
-                // request conversion
-                [=, &put](const hbase::Put& put,
-                          const std::string& region_name) -> std::unique_ptr<Request> {
-                  auto checkReq = RequestConverter::CheckAndPutToMutateRequest(
-                      row, family, qualifier, value, compare_op, put, region_name);
-                  return checkReq;
-                },
-                // response conversion
-                &ResponseConverter::BoolFromMutateResponse);
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<bool> RawAsyncTable::CheckAndDelete(const std::string& row, const std::string& family,
-                                                  const std::string& qualifier,
-                                                  const std::string& value,
-                                                  const hbase::Delete& del,
-                                                  const pb::CompareType& compare_op) {
-  auto caller =
-      CreateCallerBuilder<bool>(row, connection_conf_->write_rpc_timeout())
-          ->action([=, &del](std::shared_ptr<hbase::HBaseRpcController> controller,
-                             std::shared_ptr<hbase::RegionLocation> loc,
-                             std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<bool> {
-            return Call<hbase::Delete, hbase::Request, hbase::Response, bool>(
-                rpc_client, controller, loc, del,
-                // request conversion
-                [=, &del](const hbase::Delete& del,
-                          const std::string& region_name) -> std::unique_ptr<Request> {
-                  auto checkReq = RequestConverter::CheckAndDeleteToMutateRequest(
-                      row, family, qualifier, value, compare_op, del, region_name);
-                  return checkReq;
-                },
-                // response conversion
-                &ResponseConverter::BoolFromMutateResponse);
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<folly::Unit> RawAsyncTable::Delete(const hbase::Delete& del) {
-  auto caller =
-      CreateCallerBuilder<folly::Unit>(del.row(), connection_conf_->write_rpc_timeout())
-          ->action([=, &del](
-                       std::shared_ptr<hbase::HBaseRpcController> controller,
-                       std::shared_ptr<hbase::RegionLocation> loc,
-                       std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<folly::Unit> {
-            return Call<hbase::Delete, hbase::Request, hbase::Response, folly::Unit>(
-                rpc_client, controller, loc, del, &hbase::RequestConverter::DeleteToMutateRequest,
-                [](const Response& r) -> folly::Unit { return folly::unit; });
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<std::shared_ptr<Result>> RawAsyncTable::Append(const hbase::Append& append) {
-  auto caller =
-      CreateCallerBuilder<std::shared_ptr<Result>>(append.row(),
-                                                   connection_conf_->write_rpc_timeout())
-          ->action([=, &append](std::shared_ptr<hbase::HBaseRpcController> controller,
-                                std::shared_ptr<hbase::RegionLocation> loc,
-                                std::shared_ptr<hbase::RpcClient>
-                                    rpc_client) -> folly::Future<std::shared_ptr<Result>> {
-            return Call<hbase::Append, hbase::Request, hbase::Response, std::shared_ptr<Result>>(
-                rpc_client, controller, loc, append,
-                &hbase::RequestConverter::AppendToMutateRequest,
-                &hbase::ResponseConverter::FromMutateResponse);
-          })
-          ->Build();
-
-  return caller->Call().then([caller](const auto r) { return r; });
-}
-
-folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> RawAsyncTable::Get(
-    const std::vector<hbase::Get>& gets) {
-  std::vector<std::shared_ptr<hbase::Row>> rows;
-  for (auto get : gets) {
-    std::shared_ptr<hbase::Row> srow = std::make_shared<hbase::Get>(get);
-    rows.push_back(srow);
-  }
-  return this->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(
-      rows, connection_conf_->read_rpc_timeout());
-}
-
-template <typename REQ, typename RESP>
-folly::Future<std::vector<folly::Try<RESP>>> RawAsyncTable::Batch(
-    const std::vector<REQ>& rows, std::chrono::nanoseconds timeout) {
-  auto caller = connection_->caller_factory()
-                    ->Batch<REQ, RESP>()
-                    ->table(table_name_)
-                    ->actions(std::make_shared<std::vector<REQ>>(rows))
-                    ->rpc_timeout(timeout)
-                    ->operation_timeout(connection_conf_->operation_timeout())
-                    ->pause(connection_conf_->pause())
-                    ->max_attempts(connection_conf_->max_retries())
-                    ->start_log_errors_count(connection_conf_->start_log_errors_count())
-                    ->Build();
-
-  return caller->Call().then([caller](auto r) { return r; });
-}
-
-void RawAsyncTable::Scan(const hbase::Scan& scan, std::shared_ptr<RawScanResultConsumer> consumer) {
-  auto scanner = AsyncClientScanner::Create(
-      connection_, SetDefaultScanConfig(scan), table_name_, consumer, connection_conf_->pause(),
-      connection_conf_->max_retries(), connection_conf_->scan_timeout(),
-      connection_conf_->rpc_timeout(), connection_conf_->start_log_errors_count());
-  scanner->Start();
-}
-
-std::shared_ptr<hbase::Scan> RawAsyncTable::SetDefaultScanConfig(const hbase::Scan& scan) {
-  // always create a new scan object as we may reset the start row later.
-  auto new_scan = std::make_shared<hbase::Scan>(scan);
-  if (new_scan->Caching() <= 0) {
-    new_scan->SetCaching(default_scanner_caching_);
-  }
-  if (new_scan->MaxResultSize() <= 0) {
-    new_scan->SetMaxResultSize(default_scanner_max_result_size_);
-  }
-  return new_scan;
-}
-
-folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> RawAsyncTable::Put(
-    const std::vector<hbase::Put>& puts) {
-  std::vector<std::shared_ptr<hbase::Row>> rows;
-  for (auto put : puts) {
-    std::shared_ptr<hbase::Row> srow = std::make_shared<hbase::Put>(put);
-    rows.push_back(srow);
-  }
-  return this->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(
-      rows, connection_conf_->write_rpc_timeout());
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/raw-async-table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/raw-async-table.h b/hbase-native-client/core/raw-async-table.h
deleted file mode 100644
index 97eef7f..0000000
--- a/hbase-native-client/core/raw-async-table.h
+++ /dev/null
@@ -1,115 +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.
- *
- */
-#pragma once
-
-#include <folly/Unit.h>
-#include <folly/futures/Future.h>
-#include <chrono>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "core/async-batch-rpc-retrying-caller.h"
-#include "core/async-client-scanner.h"
-#include "core/async-connection.h"
-#include "core/async-rpc-retrying-caller-factory.h"
-#include "core/async-rpc-retrying-caller.h"
-#include "core/connection-configuration.h"
-#include "core/delete.h"
-#include "core/get.h"
-#include "core/increment.h"
-#include "core/put.h"
-#include "core/result.h"
-#include "core/scan.h"
-
-namespace hbase {
-
-/**
- * A low level asynchronous table that should not be used by user applications.The implementation
- * is required to be thread safe.
- */
-class RawAsyncTable {
- public:
-  RawAsyncTable(std::shared_ptr<pb::TableName> table_name,
-                std::shared_ptr<AsyncConnection> connection)
-      : connection_(connection),
-        connection_conf_(connection->connection_conf()),
-        table_name_(table_name),
-        rpc_client_(connection->rpc_client()) {
-    default_scanner_caching_ = connection_conf_->scanner_caching();
-    default_scanner_max_result_size_ = connection_conf_->scanner_max_result_size();
-  }
-  virtual ~RawAsyncTable() = default;
-
-  folly::Future<std::shared_ptr<Result>> Get(const hbase::Get& get);
-
-  folly::Future<folly::Unit> Delete(const hbase::Delete& del);
-
-  folly::Future<std::shared_ptr<hbase::Result>> Append(const hbase::Append& append);
-
-  folly::Future<std::shared_ptr<hbase::Result>> Increment(const hbase::Increment& increment);
-
-  folly::Future<folly::Unit> Put(const hbase::Put& put);
-
-  folly::Future<bool> CheckAndPut(const std::string& row, const std::string& family,
-                                  const std::string& qualifier, const std::string& value,
-                                  const hbase::Put& put,
-                                  const pb::CompareType& compare_op = pb::CompareType::EQUAL);
-
-  folly::Future<bool> CheckAndDelete(const std::string& row, const std::string& family,
-                                     const std::string& qualifier, const std::string& value,
-                                     const hbase::Delete& del,
-                                     const pb::CompareType& compare_op = pb::CompareType::EQUAL);
-
-  void Scan(const hbase::Scan& scan, std::shared_ptr<RawScanResultConsumer> consumer);
-
-  void Close() {}
-
-  folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> Get(
-      const std::vector<hbase::Get>& gets);
-  template <typename REQ, typename RESP>
-  folly::Future<std::vector<folly::Try<RESP>>> Batch(const std::vector<REQ>& rows,
-                                                     std::chrono::nanoseconds timeout);
-  folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> Put(
-      const std::vector<hbase::Put>& puts);
-
- private:
-  /* Data */
-  std::shared_ptr<AsyncConnection> connection_;
-  std::shared_ptr<ConnectionConfiguration> connection_conf_;
-  std::shared_ptr<pb::TableName> table_name_;
-  std::shared_ptr<RpcClient> rpc_client_;
-  int32_t default_scanner_caching_;
-  int64_t default_scanner_max_result_size_;
-
-  /* Methods */
-  template <typename REQ, typename PREQ, typename PRESP, typename RESP>
-  folly::Future<RESP> Call(
-      std::shared_ptr<RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
-      std::shared_ptr<RegionLocation> loc, const REQ& req,
-      const ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
-      const RespConverter<RESP, PRESP> resp_converter);
-
-  template <typename RESP>
-  std::shared_ptr<SingleRequestCallerBuilder<RESP>> CreateCallerBuilder(
-      std::string row, std::chrono::nanoseconds rpc_timeout);
-
-  std::shared_ptr<hbase::Scan> SetDefaultScanConfig(const hbase::Scan& scan);
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/raw-scan-result-consumer.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/raw-scan-result-consumer.h b/hbase-native-client/core/raw-scan-result-consumer.h
deleted file mode 100644
index b7c3c48..0000000
--- a/hbase-native-client/core/raw-scan-result-consumer.h
+++ /dev/null
@@ -1,131 +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.
- *
- */
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/Logging.h>
-#include <chrono>
-#include <memory>
-#include <string>
-#include <thread>
-#include <vector>
-
-#include "core/result.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-enum class ScanControllerState { kInitialized, kSuspended, kTerminated, kDestroyed };
-
-enum class ScanResumerState { kInitialized, kSuspended, kResumed };
-
-/**
- * Used to resume a scan.
- */
-class ScanResumer {
- public:
-  virtual ~ScanResumer() = default;
-
-  /**
-   * Resume the scan. You are free to call it multiple time but only the first call will take
-   * effect.
-   */
-  virtual void Resume() = 0;
-};
-
-/**
- * Used to suspend or stop a scan.
- * <p>
- * Notice that, you should only call the methods below inside onNext or onHeartbeat method. A
- * IllegalStateException will be thrown if you call them at other places.
- * <p>
- * You can only call one of the methods below, i.e., call suspend or terminate(of course you are
- * free to not call them both), and the methods are not reentrant. A IllegalStateException will be
- * thrown if you have already called one of the methods.
- */
-class ScanController {
- public:
-  virtual ~ScanController() = default;
-
-  /**
-   * Suspend the scan.
-   * <p>
-   * This means we will stop fetching data in background, i.e., will not call onNext any more
-   * before you resume the scan.
-   * @return A resumer used to resume the scan later.
-   */
-  virtual std::shared_ptr<ScanResumer> Suspend() = 0;
-
-  /**
-   * Terminate the scan.
-   * <p>
-   * This is useful when you have got enough results and want to stop the scan in onNext method,
-   * or you want to stop the scan in onHeartbeat method because it has spent too many time.
-   */
-  virtual void Terminate() = 0;
-};
-
-/**
- * Receives {@link Result} for an asynchronous scan.
- * <p>
- * Notice that, the {@link #onNext(Result[], ScanController)} method will be called in the thread
- * which we send request to HBase service. So if you want the asynchronous scanner fetch data from
- * HBase in background while you process the returned data, you need to move the processing work to
- * another thread to make the {@code onNext} call return immediately. And please do NOT do any time
- * consuming tasks in all methods below unless you know what you are doing.
- */
-class RawScanResultConsumer {
- public:
-  virtual ~RawScanResultConsumer() = default;
-
-  /**
-   * Indicate that we have receive some data.
-   * @param results the data fetched from HBase service.
-   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
-   *          instance is only valid within scope of onNext method. You can only call its method in
-   *          onNext, do NOT store it and call it later outside onNext.
-   */
-  virtual void OnNext(const std::vector<std::shared_ptr<Result>> &results,
-                      std::shared_ptr<ScanController> controller) {}
-
-  /**
-   * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
-   * onNext.
-   * <p>
-   * This method give you a chance to terminate a slow scan operation.
-   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
-   *          instance is only valid within the scope of onHeartbeat method. You can only call its
-   *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
-   */
-  virtual void OnHeartbeat(std::shared_ptr<ScanController> controller) {}
-
-  /**
-   * Indicate that we hit an unrecoverable error and the scan operation is terminated.
-   * <p>
-   * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
-   */
-  virtual void OnError(const folly::exception_wrapper &error) {}
-
-  /**
-   * Indicate that the scan operation is completed normally.
-   */
-  virtual void OnComplete() {}
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/region-location.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/region-location.h b/hbase-native-client/core/region-location.h
deleted file mode 100644
index f73999f..0000000
--- a/hbase-native-client/core/region-location.h
+++ /dev/null
@@ -1,78 +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.
- *
- */
-#pragma once
-
-#include <memory>
-#include <string>
-
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-enum class RegionLocateType { kBefore, kCurrent, kAfter };
-
-/**
- * @brief class to hold where a region is located.
- *
- * This class holds where a region is located, the information about it, the
- * region name.
- */
-class RegionLocation {
- public:
-  /**
-   * Constructor.
-   * @param region_name The region name of this region.
-   * @param ri The decoded RegionInfo of this region.
-   * @param sn The server name of the HBase regionserver thought to be hosting
-   * this region.
-   */
-  RegionLocation(std::string region_name, hbase::pb::RegionInfo ri, hbase::pb::ServerName sn)
-      : region_name_(region_name), ri_(ri), sn_(sn) {}
-
-  /**
-   * Get a reference to the regio info
-   */
-  const hbase::pb::RegionInfo &region_info() const { return ri_; }
-
-  /**
-   * Get a reference to the server name
-   */
-  const hbase::pb::ServerName &server_name() const { return sn_; }
-
-  /**
-   * Get a reference to the region name.
-   */
-  const std::string &region_name() const { return region_name_; }
-
-  /**
-   * Set the servername if the region has moved.
-   */
-  void set_server_name(hbase::pb::ServerName sn) { sn_ = sn; }
-
-  const std::string DebugString() const {
-    return "region_info:" + ri_.ShortDebugString() + ", server_name:" + sn_.ShortDebugString();
-  }
-
- private:
-  std::string region_name_;
-  hbase::pb::RegionInfo ri_;
-  hbase::pb::ServerName sn_;
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/region-request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/region-request.h b/hbase-native-client/core/region-request.h
deleted file mode 100644
index aded3a9..0000000
--- a/hbase-native-client/core/region-request.h
+++ /dev/null
@@ -1,46 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <queue>
-#include <vector>
-#include "core/action.h"
-#include "core/region-location.h"
-
-namespace hbase {
-
-class RegionRequest {
- public:
-  // Concurrent
-  using ActionList = std::vector<std::shared_ptr<Action>>;
-  explicit RegionRequest(const std::shared_ptr<hbase::RegionLocation> &region_loc)
-      : region_loc_(region_loc) {}
-  ~RegionRequest() {}
-  void AddAction(std::shared_ptr<Action> action) { actions_.push_back(action); }
-  std::shared_ptr<hbase::RegionLocation> region_location() const { return region_loc_; }
-  const ActionList &actions() const { return actions_; }
-
- private:
-  std::shared_ptr<hbase::RegionLocation> region_loc_;
-  ActionList actions_;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/region-result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/region-result.cc b/hbase-native-client/core/region-result.cc
deleted file mode 100644
index 206c876..0000000
--- a/hbase-native-client/core/region-result.cc
+++ /dev/null
@@ -1,53 +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/region-result.h"
-#include <glog/logging.h>
-#include <stdexcept>
-
-using hbase::pb::RegionLoadStats;
-
-namespace hbase {
-
-RegionResult::RegionResult() {}
-
-RegionResult::~RegionResult() {}
-
-void RegionResult::AddResultOrException(int32_t index, std::shared_ptr<hbase::Result> result,
-                                        std::shared_ptr<folly::exception_wrapper> exc) {
-  auto index_found = result_or_excption_.find(index);
-  if (index_found == result_or_excption_.end()) {
-    result_or_excption_[index] = std::make_tuple(result ? result : nullptr, exc ? exc : nullptr);
-  } else {
-    throw std::runtime_error("Index " + std::to_string(index) +
-                             " already set with ResultOrException");
-  }
-}
-
-void RegionResult::set_stat(std::shared_ptr<RegionLoadStats> stat) { stat_ = stat; }
-
-int RegionResult::ResultOrExceptionSize() const { return result_or_excption_.size(); }
-
-std::shared_ptr<ResultOrExceptionTuple> RegionResult::ResultOrException(int32_t index) const {
-  return std::make_shared<ResultOrExceptionTuple>(result_or_excption_.at(index));
-}
-
-const std::shared_ptr<RegionLoadStats>& RegionResult::stat() const { return stat_; }
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/region-result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/region-result.h b/hbase-native-client/core/region-result.h
deleted file mode 100644
index b961634..0000000
--- a/hbase-native-client/core/region-result.h
+++ /dev/null
@@ -1,55 +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.
- *
- */
-
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <map>
-#include <memory>
-#include <string>
-#include <tuple>
-#include "core/result.h"
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-using ResultOrExceptionTuple =
-    std::tuple<std::shared_ptr<hbase::Result>, std::shared_ptr<folly::exception_wrapper>>;
-
-class RegionResult {
- public:
-  RegionResult();
-  void AddResultOrException(int32_t index, std::shared_ptr<hbase::Result> result,
-                            std::shared_ptr<folly::exception_wrapper> exc);
-
-  void set_stat(std::shared_ptr<pb::RegionLoadStats> stat);
-
-  int ResultOrExceptionSize() const;
-
-  std::shared_ptr<ResultOrExceptionTuple> ResultOrException(int32_t index) const;
-
-  const std::shared_ptr<pb::RegionLoadStats>& stat() const;
-
-  ~RegionResult();
-
- private:
-  std::map<int, ResultOrExceptionTuple> result_or_excption_;
-  std::shared_ptr<pb::RegionLoadStats> stat_;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/request-converter-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/request-converter-test.cc b/hbase-native-client/core/request-converter-test.cc
deleted file mode 100644
index 6c07a19..0000000
--- a/hbase-native-client/core/request-converter-test.cc
+++ /dev/null
@@ -1,126 +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/request-converter.h"
-
-#include <gtest/gtest.h>
-#include <limits>
-#include "connection/request.h"
-#include "core/get.h"
-#include "core/scan.h"
-
-using hbase::Get;
-using hbase::Scan;
-
-using hbase::pb::GetRequest;
-using hbase::pb::RegionSpecifier;
-using hbase::pb::RegionSpecifier_RegionSpecifierType;
-using hbase::pb::ScanRequest;
-
-TEST(RequestConverter, ToGet) {
-  std::string row_str = "row-test";
-  Get get(row_str);
-  get.AddFamily("family-1");
-  get.AddFamily("family-2");
-  get.AddFamily("family-3");
-  get.AddColumn("family-2", "qualifier-1");
-  get.AddColumn("family-2", "qualifier-2");
-  get.AddColumn("family-2", "qualifier-3");
-  get.SetCacheBlocks(false);
-  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
-  get.SetMaxVersions(2);
-  get.SetTimeRange(10000, 20000);
-  std::string region_name("RegionName");
-
-  auto req = hbase::RequestConverter::ToGetRequest(get, region_name);
-  auto msg = std::static_pointer_cast<GetRequest>(req->req_msg());
-
-  // Tests whether the PB object is properly set or not.
-  ASSERT_TRUE(msg->has_region());
-  ASSERT_TRUE(msg->region().has_value());
-  EXPECT_EQ(msg->region().value(), region_name);
-
-  ASSERT_TRUE(msg->has_get());
-  EXPECT_EQ(msg->get().row(), row_str);
-  EXPECT_FALSE(msg->get().cache_blocks());
-  EXPECT_EQ(msg->get().consistency(), hbase::pb::Consistency::TIMELINE);
-  EXPECT_EQ(msg->get().max_versions(), 2);
-  EXPECT_EQ(msg->get().column_size(), 3);
-  for (int i = 0; i < msg->get().column_size(); ++i) {
-    EXPECT_EQ(msg->get().column(i).family(), "family-" + std::to_string(i + 1));
-    for (int j = 0; j < msg->get().column(i).qualifier_size(); ++j) {
-      EXPECT_EQ(msg->get().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
-    }
-  }
-}
-
-TEST(RequestConverter, ToScan) {
-  std::string start_row("start-row");
-  std::string stop_row("stop-row");
-  hbase::Scan scan;
-  scan.AddFamily("family-1");
-  scan.AddFamily("family-2");
-  scan.AddFamily("family-3");
-  scan.AddColumn("family-2", "qualifier-1");
-  scan.AddColumn("family-2", "qualifier-2");
-  scan.AddColumn("family-2", "qualifier-3");
-  scan.SetReversed(true);
-  scan.SetStartRow(start_row);
-  scan.SetStopRow(stop_row);
-  scan.SetCaching(3);
-  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
-  scan.SetCacheBlocks(true);
-  scan.SetAllowPartialResults(true);
-  scan.SetLoadColumnFamiliesOnDemand(true);
-  scan.SetMaxVersions(5);
-  scan.SetTimeRange(10000, 20000);
-  std::string region_name("RegionName");
-
-  auto req = hbase::RequestConverter::ToScanRequest(scan, region_name);
-  auto msg = std::static_pointer_cast<ScanRequest>(req->req_msg());
-
-  // Tests whether the PB object is properly set or not.
-  ASSERT_TRUE(msg->has_region());
-  ASSERT_TRUE(msg->region().has_value());
-  EXPECT_EQ(msg->region().value(), region_name);
-
-  ASSERT_TRUE(msg->has_scan());
-  EXPECT_TRUE(msg->scan().reversed());
-  EXPECT_EQ(msg->scan().start_row(), start_row);
-  EXPECT_EQ(msg->scan().stop_row(), stop_row);
-  EXPECT_FALSE(msg->scan().small());
-  EXPECT_EQ(msg->scan().caching(), 3);
-  EXPECT_EQ(msg->scan().consistency(), hbase::pb::Consistency::TIMELINE);
-  EXPECT_TRUE(msg->scan().cache_blocks());
-  EXPECT_TRUE(msg->scan().allow_partial_results());
-  EXPECT_TRUE(msg->scan().load_column_families_on_demand());
-  EXPECT_EQ(msg->scan().max_versions(), 5);
-  EXPECT_EQ(msg->scan().max_result_size(), std::numeric_limits<uint64_t>::max());
-
-  EXPECT_EQ(msg->scan().column_size(), 3);
-  for (int i = 0; i < msg->scan().column_size(); ++i) {
-    EXPECT_EQ(msg->scan().column(i).family(), "family-" + std::to_string(i + 1));
-    for (int j = 0; j < msg->scan().column(i).qualifier_size(); ++j) {
-      EXPECT_EQ(msg->scan().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
-    }
-  }
-  ASSERT_FALSE(msg->client_handles_partials());
-  ASSERT_FALSE(msg->client_handles_heartbeats());
-  ASSERT_FALSE(msg->track_scan_metrics());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/request-converter.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/request-converter.cc b/hbase-native-client/core/request-converter.cc
deleted file mode 100644
index f48f228..0000000
--- a/hbase-native-client/core/request-converter.cc
+++ /dev/null
@@ -1,368 +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/request-converter.h"
-
-#include <folly/Conv.h>
-
-#include <utility>
-#include "if/Client.pb.h"
-
-using hbase::pb::GetRequest;
-using hbase::pb::MutationProto;
-using hbase::pb::RegionAction;
-using hbase::pb::RegionSpecifier;
-using hbase::pb::RegionSpecifier_RegionSpecifierType;
-using hbase::pb::ScanRequest;
-
-namespace hbase {
-
-RequestConverter::~RequestConverter() {}
-
-RequestConverter::RequestConverter() {}
-
-void RequestConverter::SetRegion(const std::string &region_name,
-                                 RegionSpecifier *region_specifier) {
-  region_specifier->set_type(
-      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_REGION_NAME);
-  region_specifier->set_value(region_name);
-}
-
-std::unique_ptr<Request> RequestConverter::ToGetRequest(const Get &get,
-                                                        const std::string &region_name) {
-  auto pb_req = Request::get();
-  auto pb_msg = std::static_pointer_cast<GetRequest>(pb_req->req_msg());
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-  pb_msg->set_allocated_get((RequestConverter::ToGet(get)).release());
-  return pb_req;
-}
-
-std::unique_ptr<hbase::pb::Scan> RequestConverter::ToScan(const Scan &scan) {
-  auto pb_scan = std::make_unique<hbase::pb::Scan>();
-  pb_scan->set_max_versions(scan.MaxVersions());
-  pb_scan->set_cache_blocks(scan.CacheBlocks());
-  pb_scan->set_reversed(scan.IsReversed());
-  pb_scan->set_caching(scan.Caching());
-  pb_scan->set_start_row(scan.StartRow());
-  pb_scan->set_stop_row(scan.StopRow());
-  pb_scan->set_consistency(scan.Consistency());
-  pb_scan->set_max_result_size(scan.MaxResultSize());
-  pb_scan->set_allow_partial_results(scan.AllowPartialResults());
-  pb_scan->set_load_column_families_on_demand(scan.LoadColumnFamiliesOnDemand());
-
-  if (!scan.Timerange().IsAllTime()) {
-    hbase::pb::TimeRange *pb_time_range = pb_scan->mutable_time_range();
-    pb_time_range->set_from(scan.Timerange().MinTimeStamp());
-    pb_time_range->set_to(scan.Timerange().MaxTimeStamp());
-  }
-
-  if (scan.HasFamilies()) {
-    for (const auto &family : scan.FamilyMap()) {
-      auto column = pb_scan->add_column();
-      column->set_family(family.first);
-      for (const auto &qualifier : family.second) {
-        column->add_qualifier(qualifier);
-      }
-    }
-  }
-
-  if (scan.filter() != nullptr) {
-    pb_scan->set_allocated_filter(Filter::ToProto(*(scan.filter())).release());
-  }
-
-  return std::move(pb_scan);
-}
-
-std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
-                                                         const std::string &region_name) {
-  auto pb_req = Request::scan();
-  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
-
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_scan(ToScan(scan).release());
-
-  SetCommonScanRequestFields(pb_msg, false);
-
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
-                                                         const std::string &region_name,
-                                                         int32_t num_rows, bool close_scanner) {
-  auto pb_req = Request::scan();
-  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
-
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_scan(ToScan(scan).release());
-
-  pb_msg->set_number_of_rows(num_rows);
-  pb_msg->set_close_scanner(close_scanner);
-
-  SetCommonScanRequestFields(pb_msg, false);
-
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::ToScanRequest(int64_t scanner_id, int32_t num_rows,
-                                                         bool close_scanner) {
-  auto pb_req = Request::scan();
-  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
-
-  pb_msg->set_number_of_rows(num_rows);
-  pb_msg->set_close_scanner(close_scanner);
-  pb_msg->set_scanner_id(scanner_id);
-
-  SetCommonScanRequestFields(pb_msg, false);
-
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::ToScanRequest(int64_t scanner_id, int32_t num_rows,
-                                                         bool close_scanner,
-                                                         int64_t next_call_seq_id, bool renew) {
-  auto pb_req = Request::scan();
-  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
-
-  pb_msg->set_number_of_rows(num_rows);
-  pb_msg->set_close_scanner(close_scanner);
-  pb_msg->set_scanner_id(scanner_id);
-  pb_msg->set_next_call_seq(next_call_seq_id);
-
-  SetCommonScanRequestFields(pb_msg, renew);
-  return pb_req;
-}
-
-void RequestConverter::SetCommonScanRequestFields(std::shared_ptr<hbase::pb::ScanRequest> pb_msg,
-                                                  bool renew) {
-  // TODO We will change these later when we implement partial results and heartbeats, etc
-  pb_msg->set_client_handles_partials(false);
-  pb_msg->set_client_handles_heartbeats(false);
-  pb_msg->set_track_scan_metrics(false);
-  pb_msg->set_renew(renew);
-  // TODO: set scan limit
-}
-
-std::unique_ptr<Request> RequestConverter::ToMultiRequest(
-    const ActionsByRegion &actions_by_region) {
-  auto pb_req = Request::multi();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MultiRequest>(pb_req->req_msg());
-
-  for (const auto &action_by_region : actions_by_region) {
-    auto pb_region_action = pb_msg->add_regionaction();
-    RequestConverter::SetRegion(action_by_region.first, pb_region_action->mutable_region());
-    int action_num = 0;
-    for (const auto &region_action : action_by_region.second->actions()) {
-      auto pb_action = pb_region_action->add_action();
-      auto pget = region_action->action();
-      // We store only hbase::Get in hbase::Action as of now. It will be changed later on.
-      CHECK(pget) << "Unexpected. action can't be null.";
-      std::string error_msg("");
-      if (typeid(*pget) == typeid(hbase::Get)) {
-        auto getp = dynamic_cast<hbase::Get *>(pget.get());
-        pb_action->set_allocated_get(RequestConverter::ToGet(*getp).release());
-      } else if (typeid(*pget) == typeid(hbase::Put)) {
-        auto putp = dynamic_cast<hbase::Put *>(pget.get());
-        pb_action->set_allocated_mutation(
-            RequestConverter::ToMutation(MutationType::MutationProto_MutationType_PUT, *putp, -1)
-                .release());
-      } else {
-        throw std::runtime_error("Unexpected action type encountered.");
-      }
-      pb_action->set_index(action_num);
-      action_num++;
-    }
-  }
-  return pb_req;
-}
-
-std::unique_ptr<hbase::pb::Get> RequestConverter::ToGet(const Get &get) {
-  auto pb_get = std::make_unique<hbase::pb::Get>();
-  pb_get->set_max_versions(get.MaxVersions());
-  pb_get->set_cache_blocks(get.CacheBlocks());
-  pb_get->set_consistency(get.Consistency());
-
-  if (!get.Timerange().IsAllTime()) {
-    hbase::pb::TimeRange *pb_time_range = pb_get->mutable_time_range();
-    pb_time_range->set_from(get.Timerange().MinTimeStamp());
-    pb_time_range->set_to(get.Timerange().MaxTimeStamp());
-  }
-  pb_get->set_row(get.row());
-  if (get.HasFamilies()) {
-    for (const auto &family : get.FamilyMap()) {
-      auto column = pb_get->add_column();
-      column->set_family(family.first);
-      for (const auto &qualifier : family.second) {
-        column->add_qualifier(qualifier);
-      }
-    }
-  }
-
-  if (get.filter() != nullptr) {
-    pb_get->set_allocated_filter(Filter::ToProto(*(get.filter())).release());
-  }
-  return pb_get;
-}
-
-std::unique_ptr<MutationProto> RequestConverter::ToMutation(const MutationType type,
-                                                            const Mutation &mutation,
-                                                            const int64_t nonce) {
-  auto pb_mut = std::make_unique<MutationProto>();
-  pb_mut->set_row(mutation.row());
-  pb_mut->set_mutate_type(type);
-  pb_mut->set_durability(mutation.Durability());
-  pb_mut->set_timestamp(mutation.TimeStamp());
-  // TODO: set attributes from the mutation (key value pairs).
-
-  if (nonce > 0) {
-    pb_mut->set_nonce(nonce);
-  }
-
-  for (const auto &family : mutation.FamilyMap()) {
-    for (const auto &cell : family.second) {
-      auto column = pb_mut->add_column_value();
-      column->set_family(cell->Family());
-      auto qual = column->add_qualifier_value();
-      qual->set_qualifier(cell->Qualifier());
-      qual->set_timestamp(cell->Timestamp());
-      auto cell_type = cell->Type();
-      if (type == pb::MutationProto_MutationType_DELETE ||
-          (type == pb::MutationProto_MutationType_PUT && IsDelete(cell_type))) {
-        qual->set_delete_type(ToDeleteType(cell_type));
-      }
-
-      qual->set_value(cell->Value());
-    }
-  }
-  return std::move(pb_mut);
-}
-
-DeleteType RequestConverter::ToDeleteType(const CellType type) {
-  switch (type) {
-    case CellType::DELETE:
-      return pb::MutationProto_DeleteType_DELETE_ONE_VERSION;
-    case CellType::DELETE_COLUMN:
-      return pb::MutationProto_DeleteType_DELETE_MULTIPLE_VERSIONS;
-    case CellType::DELETE_FAMILY:
-      return pb::MutationProto_DeleteType_DELETE_FAMILY;
-    case CellType::DELETE_FAMILY_VERSION:
-      return pb::MutationProto_DeleteType_DELETE_FAMILY_VERSION;
-    default:
-      throw std::runtime_error("Unknown delete type: " + folly::to<std::string>(type));
-  }
-}
-
-bool RequestConverter::IsDelete(const CellType type) {
-  return CellType::DELETE <= type && type <= CellType::DELETE_FAMILY;
-}
-
-std::unique_ptr<Request> RequestConverter::ToMutateRequest(const Put &put,
-                                                           const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_PUT, put, -1).release());
-
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::CheckAndPutToMutateRequest(
-    const std::string &row, const std::string &family, const std::string &qualifier,
-    const std::string &value, const pb::CompareType compare_op, const hbase::Put &put,
-    const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_PUT, put, -1).release());
-  ::hbase::pb::Condition *cond = pb_msg->mutable_condition();
-  cond->set_row(row);
-  cond->set_family(family);
-  cond->set_qualifier(qualifier);
-  cond->set_allocated_comparator(
-      Comparator::ToProto(*(ComparatorFactory::BinaryComparator(value).get())).release());
-  cond->set_compare_type(compare_op);
-
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::CheckAndDeleteToMutateRequest(
-    const std::string &row, const std::string &family, const std::string &qualifier,
-    const std::string &value, const pb::CompareType compare_op, const hbase::Delete &del,
-    const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_DELETE, del, -1).release());
-  ::hbase::pb::Condition *cond = pb_msg->mutable_condition();
-  cond->set_row(row);
-  cond->set_family(family);
-  cond->set_qualifier(qualifier);
-  cond->set_allocated_comparator(
-      Comparator::ToProto(*(ComparatorFactory::BinaryComparator(value).get())).release());
-  cond->set_compare_type(compare_op);
-
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::DeleteToMutateRequest(const Delete &del,
-                                                                 const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_DELETE, del, -1).release());
-
-  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
-  return pb_req;
-}
-std::unique_ptr<Request> RequestConverter::IncrementToMutateRequest(
-    const Increment &incr, const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_INCREMENT, incr, -1).release());
-
-  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
-  return pb_req;
-}
-
-std::unique_ptr<Request> RequestConverter::AppendToMutateRequest(const Append &append,
-                                                                 const std::string &region_name) {
-  auto pb_req = Request::mutate();
-  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
-  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
-
-  pb_msg->set_allocated_mutation(
-      ToMutation(MutationType::MutationProto_MutationType_APPEND, append, -1).release());
-
-  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
-  return pb_req;
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/request-converter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/request-converter.h b/hbase-native-client/core/request-converter.h
deleted file mode 100644
index bcea278..0000000
--- a/hbase-native-client/core/request-converter.h
+++ /dev/null
@@ -1,125 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <string>
-#include <vector>
-#include "connection/request.h"
-#include "core/action.h"
-#include "core/append.h"
-#include "core/cell.h"
-#include "core/delete.h"
-#include "core/get.h"
-#include "core/increment.h"
-#include "core/mutation.h"
-#include "core/put.h"
-#include "core/region-request.h"
-#include "core/scan.h"
-#include "core/server-request.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-
-using MutationType = hbase::pb::MutationProto_MutationType;
-using DeleteType = hbase::pb::MutationProto_DeleteType;
-
-namespace hbase {
-
-using ActionsByRegion = ServerRequest::ActionsByRegion;
-/**
- * RequestConverter class
- * This class converts a Client side Get, Scan, Mutate operation to corresponding PB message.
- */
-class RequestConverter {
- public:
-  ~RequestConverter();
-
-  /**
-   * @brief Returns a Request object comprising of PB GetRequest created using
-   * passed 'get'
-   * @param get - Get object used for creating GetRequest
-   * @param region_name - table region
-   */
-  static std::unique_ptr<Request> ToGetRequest(const Get &get, const std::string &region_name);
-
-  /**
-   * @brief Returns a Request object comprising of PB ScanRequest created using
-   * passed 'scan'
-   * @param scan - Scan object used for creating ScanRequest
-   * @param region_name - table region
-   */
-  static std::unique_ptr<Request> ToScanRequest(const Scan &scan, const std::string &region_name);
-
-  static std::unique_ptr<Request> ToScanRequest(const Scan &scan, const std::string &region_name,
-                                                int32_t num_rows, bool close_scanner);
-
-  static std::unique_ptr<Request> ToScanRequest(int64_t scanner_id, int32_t num_rows,
-                                                bool close_scanner);
-
-  static std::unique_ptr<Request> ToScanRequest(int64_t scanner_id, int32_t num_rows,
-                                                bool close_scanner, int64_t next_call_seq_id,
-                                                bool renew);
-
-  static std::unique_ptr<Request> ToMultiRequest(const ActionsByRegion &region_requests);
-
-  static std::unique_ptr<Request> DeleteToMutateRequest(const Delete &del,
-                                                        const std::string &region_name);
-
-  static std::unique_ptr<Request> ToMutateRequest(const Put &put, const std::string &region_name);
-
-  static std::unique_ptr<Request> CheckAndPutToMutateRequest(
-      const std::string &row, const std::string &family, const std::string &qualifier,
-      const std::string &value, const pb::CompareType compare_op, const hbase::Put &put,
-      const std::string &region_name);
-
-  static std::unique_ptr<Request> CheckAndDeleteToMutateRequest(
-      const std::string &row, const std::string &family, const std::string &qualifier,
-      const std::string &value, const pb::CompareType compare_op, const hbase::Delete &del,
-      const std::string &region_name);
-
-  static std::unique_ptr<Request> IncrementToMutateRequest(const Increment &incr,
-                                                           const std::string &region_name);
-
-  static std::unique_ptr<pb::MutationProto> ToMutation(const MutationType type,
-                                                       const Mutation &mutation,
-                                                       const int64_t nonce);
-
-  static std::unique_ptr<Request> AppendToMutateRequest(const Append &append,
-                                                        const std::string &region_name);
-
- private:
-  // Constructor not required. We have all static methods to create PB requests.
-  RequestConverter();
-
-  /**
-   * @brief fills region_specifier with region values.
-   * @param region_name - table region
-   * @param region_specifier - RegionSpecifier to be filled and passed in PB
-   * Request.
-   */
-  static void SetRegion(const std::string &region_name, pb::RegionSpecifier *region_specifier);
-  static std::unique_ptr<hbase::pb::Get> ToGet(const Get &get);
-  static std::unique_ptr<hbase::pb::Scan> ToScan(const Scan &scan);
-  static DeleteType ToDeleteType(const CellType type);
-  static bool IsDelete(const CellType type);
-  static void SetCommonScanRequestFields(std::shared_ptr<hbase::pb::ScanRequest>, bool renew);
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/response-converter.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/response-converter.cc b/hbase-native-client/core/response-converter.cc
deleted file mode 100644
index 960c487..0000000
--- a/hbase-native-client/core/response-converter.cc
+++ /dev/null
@@ -1,221 +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/response-converter.h"
-#include <glog/logging.h>
-#include <stdexcept>
-#include <string>
-#include <utility>
-#include <vector>
-#include "core/cell.h"
-#include "core/multi-response.h"
-#include "exceptions/exception.h"
-
-using hbase::pb::GetResponse;
-using hbase::pb::MutateResponse;
-using hbase::pb::ScanResponse;
-using hbase::pb::RegionLoadStats;
-
-namespace hbase {
-
-ResponseConverter::ResponseConverter() {}
-
-ResponseConverter::~ResponseConverter() {}
-
-// impl note: we are returning shared_ptr's instead of unique_ptr's because these
-// go inside folly::Future's, making the move semantics extremely tricky.
-std::shared_ptr<Result> ResponseConverter::FromGetResponse(const Response& resp) {
-  auto get_resp = std::static_pointer_cast<GetResponse>(resp.resp_msg());
-  VLOG(3) << "FromGetResponse:" << get_resp->ShortDebugString();
-  return ToResult(get_resp->result(), resp.cell_scanner());
-}
-
-std::shared_ptr<Result> ResponseConverter::FromMutateResponse(const Response& resp) {
-  auto mutate_resp = std::static_pointer_cast<MutateResponse>(resp.resp_msg());
-  hbase::pb::Result result = mutate_resp->result();
-  return ToResult(mutate_resp->result(), resp.cell_scanner());
-}
-
-bool ResponseConverter::BoolFromMutateResponse(const Response& resp) {
-  auto mutate_resp = std::static_pointer_cast<MutateResponse>(resp.resp_msg());
-  return mutate_resp->processed();
-}
-
-std::shared_ptr<Result> ResponseConverter::ToResult(
-    const hbase::pb::Result& result, const std::shared_ptr<CellScanner> cell_scanner) {
-  std::vector<std::shared_ptr<Cell>> vcells;
-  for (auto cell : result.cell()) {
-    std::shared_ptr<Cell> pcell =
-        std::make_shared<Cell>(cell.row(), cell.family(), cell.qualifier(), cell.timestamp(),
-                               cell.value(), static_cast<hbase::CellType>(cell.cell_type()));
-    vcells.push_back(pcell);
-  }
-
-  // iterate over the cells coming from rpc codec
-  if (cell_scanner != nullptr) {
-    int cells_read = 0;
-    while (cells_read != result.associated_cell_count()) {
-      if (cell_scanner->Advance()) {
-        vcells.push_back(cell_scanner->Current());
-        cells_read += 1;
-      } else {
-        LOG(ERROR) << "CellScanner::Advance() returned false unexpectedly. Cells Read:- "
-                   << cells_read << "; Expected Cell Count:- " << result.associated_cell_count();
-        std::runtime_error("CellScanner::Advance() returned false unexpectedly");
-      }
-    }
-  }
-  return std::make_shared<Result>(vcells, result.exists(), result.stale(), result.partial());
-}
-
-std::vector<std::shared_ptr<Result>> ResponseConverter::FromScanResponse(const Response& resp) {
-  auto scan_resp = std::static_pointer_cast<ScanResponse>(resp.resp_msg());
-  return FromScanResponse(scan_resp, resp.cell_scanner());
-}
-
-std::vector<std::shared_ptr<Result>> ResponseConverter::FromScanResponse(
-    const std::shared_ptr<ScanResponse> scan_resp, std::shared_ptr<CellScanner> cell_scanner) {
-  VLOG(3) << "FromScanResponse:" << scan_resp->ShortDebugString()
-          << " cell_scanner:" << (cell_scanner != nullptr);
-  int num_results =
-      cell_scanner != nullptr ? scan_resp->cells_per_result_size() : scan_resp->results_size();
-
-  std::vector<std::shared_ptr<Result>> results{static_cast<size_t>(num_results)};
-  for (int i = 0; i < num_results; i++) {
-    if (cell_scanner != nullptr) {
-      // Cells are out in cellblocks.  Group them up again as Results.  How many to read at a
-      // time will be found in getCellsLength -- length here is how many Cells in the i'th Result
-      int num_cells = scan_resp->cells_per_result(i);
-
-      std::vector<std::shared_ptr<Cell>> vcells;
-      for (int j = 0; j < num_cells; j++) {
-        if (!cell_scanner->Advance()) {
-          std::string msg = "Results sent from server=" + std::to_string(num_results) +
-                            ". But only got " + std::to_string(i) +
-                            " results completely at client. Resetting the scanner to scan again.";
-          LOG(ERROR) << msg;
-          throw std::runtime_error(msg);
-        }
-        vcells.push_back(cell_scanner->Current());
-      }
-      // TODO: handle partial results per Result by checking partial_flag_per_result
-      results[i] = std::make_shared<Result>(vcells, false, scan_resp->stale(), false);
-    } else {
-      results[i] = ToResult(scan_resp->results(i), cell_scanner);
-    }
-  }
-
-  return results;
-}
-
-std::unique_ptr<hbase::MultiResponse> ResponseConverter::GetResults(
-    std::shared_ptr<Request> req, const Response& resp,
-    const ServerRequest::ActionsByRegion& actions_by_region) {
-  auto multi_req = std::static_pointer_cast<hbase::pb::MultiRequest>(req->req_msg());
-  auto multi_resp = std::static_pointer_cast<hbase::pb::MultiResponse>(resp.resp_msg());
-  VLOG(3) << "GetResults:" << multi_resp->ShortDebugString();
-  int req_region_action_count = multi_req->regionaction_size();
-  int res_region_action_count = multi_resp->regionactionresult_size();
-  if (req_region_action_count != res_region_action_count) {
-    throw std::runtime_error("Request mutation count=" + std::to_string(req_region_action_count) +
-                             " does not match response mutation result count=" +
-                             std::to_string(res_region_action_count));
-  }
-  auto multi_response = std::make_unique<hbase::MultiResponse>();
-  for (int32_t num = 0; num < res_region_action_count; num++) {
-    hbase::pb::RegionAction actions = multi_req->regionaction(num);
-    hbase::pb::RegionActionResult action_result = multi_resp->regionactionresult(num);
-    hbase::pb::RegionSpecifier rs = actions.region();
-    if (rs.has_type() && rs.type() != hbase::pb::RegionSpecifier::REGION_NAME) {
-      throw std::runtime_error("We support only encoded types for protobuf multi response.");
-    }
-
-    auto region_name = rs.value();
-    if (action_result.has_exception()) {
-      auto ew = ResponseConverter::GetRemoteException(action_result.exception());
-      VLOG(8) << "Store Remote Region Exception:- " << ew->what().toStdString() << "; Region["
-              << region_name << "];";
-      multi_response->AddRegionException(region_name, ew);
-      continue;
-    }
-
-    if (actions.action_size() != action_result.resultorexception_size()) {
-      throw std::runtime_error("actions.action_size=" + std::to_string(actions.action_size()) +
-                               ", action_result.resultorexception_size=" +
-                               std::to_string(action_result.resultorexception_size()) +
-                               " for region " + actions.region().value());
-    }
-
-    auto multi_actions = actions_by_region.at(region_name)->actions();
-    uint64_t multi_actions_num = 0;
-    for (hbase::pb::ResultOrException roe : action_result.resultorexception()) {
-      std::shared_ptr<Result> result;
-      std::shared_ptr<folly::exception_wrapper> ew;
-      if (roe.has_exception()) {
-        auto ew = ResponseConverter::GetRemoteException(roe.exception());
-        VLOG(8) << "Store Remote Region Exception:- " << ew->what().toStdString() << "; Region["
-                << region_name << "];";
-        multi_response->AddRegionException(region_name, ew);
-      } else if (roe.has_result()) {
-        result = ToResult(roe.result(), resp.cell_scanner());
-      } else if (roe.has_service_result()) {
-        // TODO Not processing Coprocessor Service Result;
-      } else {
-        // Sometimes, the response is just "it was processed". Generally, this occurs for things
-        // like mutateRows where either we get back 'processed' (or not) and optionally some
-        // statistics about the regions we touched.
-        std::vector<std::shared_ptr<Cell>> empty_cells;
-        result = std::make_shared<Result>(empty_cells, multi_resp->processed() ? true : false,
-                                          false, false);
-      }
-      // We add the original index of the multi-action so that when populating the response back we
-      // do it as per the action index
-      multi_response->AddRegionResult(
-          region_name, multi_actions[multi_actions_num]->original_index(), std::move(result), ew);
-      multi_actions_num++;
-    }
-  }
-
-  if (multi_resp->has_regionstatistics()) {
-    hbase::pb::MultiRegionLoadStats stats = multi_resp->regionstatistics();
-    for (int i = 0; i < stats.region_size(); i++) {
-      multi_response->AddStatistic(stats.region(i).value(),
-                                   std::make_shared<RegionLoadStats>(stats.stat(i)));
-    }
-  }
-  return multi_response;
-}
-
-std::shared_ptr<folly::exception_wrapper> ResponseConverter::GetRemoteException(
-    const hbase::pb::NameBytesPair& exc_resp) {
-  std::string what;
-  std::string exception_class_name = exc_resp.has_name() ? exc_resp.name() : "";
-  std::string stack_trace = exc_resp.has_value() ? exc_resp.value() : "";
-
-  what.append(exception_class_name).append(stack_trace);
-  auto remote_exception = std::make_unique<RemoteException>(what);
-  remote_exception->set_exception_class_name(exception_class_name)
-      ->set_stack_trace(stack_trace)
-      ->set_hostname("")
-      ->set_port(0);
-
-  return std::make_shared<folly::exception_wrapper>(
-      folly::make_exception_wrapper<RemoteException>(*remote_exception));
-}
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/response-converter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/response-converter.h b/hbase-native-client/core/response-converter.h
deleted file mode 100644
index edd4165..0000000
--- a/hbase-native-client/core/response-converter.h
+++ /dev/null
@@ -1,71 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <vector>
-#include "connection/request.h"
-#include "connection/response.h"
-#include "core/multi-response.h"
-#include "core/result.h"
-#include "core/server-request.h"
-#include "if/Client.pb.h"
-#include "serde/cell-scanner.h"
-
-namespace hbase {
-
-/**
- * ResponseConverter class
- * This class converts a PB Response to corresponding Result or other objects.
- */
-class ResponseConverter {
- public:
-  ~ResponseConverter();
-
-  static std::shared_ptr<Result> ToResult(const hbase::pb::Result& result,
-                                          const std::shared_ptr<CellScanner> cell_scanner);
-
-  /**
-   * @brief Returns a Result object created by PB Message in passed Response object.
-   * @param resp - Response object having the PB message.
-   */
-  static std::shared_ptr<hbase::Result> FromGetResponse(const Response& resp);
-
-  static std::shared_ptr<hbase::Result> FromMutateResponse(const Response& resp);
-
-  static bool BoolFromMutateResponse(const Response& resp);
-
-  static std::vector<std::shared_ptr<Result>> FromScanResponse(const Response& resp);
-
-  static std::vector<std::shared_ptr<Result>> FromScanResponse(
-      const std::shared_ptr<pb::ScanResponse> resp, std::shared_ptr<CellScanner> cell_scanner);
-
-  static std::unique_ptr<hbase::MultiResponse> GetResults(
-      std::shared_ptr<Request> req, const Response& resp,
-      const ServerRequest::ActionsByRegion& actions_by_region);
-
- private:
-  // Constructor not required. We have all static methods to extract response from PB messages.
-  ResponseConverter();
-  static std::shared_ptr<folly::exception_wrapper> GetRemoteException(
-      const hbase::pb::NameBytesPair& exc_resp);
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/result-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result-scanner.h b/hbase-native-client/core/result-scanner.h
deleted file mode 100644
index 9460521..0000000
--- a/hbase-native-client/core/result-scanner.h
+++ /dev/null
@@ -1,47 +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.
- *
- */
-
-#pragma once
-
-#include <functional>
-#include <iterator>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "core/cell.h"
-#include "core/result.h"
-
-namespace hbase {
-
-/**
- * Interface for client-side scanning. Use Table to obtain instances.
- */
-class ResultScanner {
-  // TODO: should we implement forward iterators?
-
- public:
-  virtual ~ResultScanner() {}
-
-  virtual void Close() = 0;
-
-  virtual std::shared_ptr<Result> Next() = 0;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/result-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result-test.cc b/hbase-native-client/core/result-test.cc
deleted file mode 100644
index dd60aeb..0000000
--- a/hbase-native-client/core/result-test.cc
+++ /dev/null
@@ -1,322 +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 <glog/logging.h>
-#include <gtest/gtest.h>
-#include <limits>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "core/cell.h"
-#include "core/result.h"
-
-using hbase::Cell;
-using hbase::CellType;
-using hbase::Result;
-using std::experimental::nullopt;
-
-void PopulateCells(std::vector<std::shared_ptr<Cell> > &cells) {
-  // Populate some Results
-  // We assume that for a single Cell, the corresponding row, families and
-  // qualifiers are present.
-  // We have also considered different versions in the test for the same row.
-  std::string row = "row";
-  for (int i = 0; i < 10; i++) {
-    std::string family = "family-" + std::to_string(i);
-    std::string column = "column-" + std::to_string(i);
-    std::string value = "value-" + std::to_string(i);
-
-    switch (i) {
-      case 5: {
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482113040506, "value-5", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
-        break;
-      }
-      case 8: {
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482113040506, "value-8", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482110969958, "value-Y", CellType::PUT));
-        break;
-      }
-      case 9: {
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482113040506, "value-9", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482110969958, "value-Y", CellType::PUT));
-        cells.push_back(
-            std::make_shared<Cell>(row, family, column, 1482110876075, "value-Z", CellType::PUT));
-        break;
-      }
-      default: {
-        cells.push_back(std::make_shared<Cell>(
-            row, family, column, std::numeric_limits<int64_t>::max(), value, CellType::PUT));
-      }
-    }
-  }
-  return;
-}
-
-TEST(Result, EmptyResult) {
-  std::vector<std::shared_ptr<Cell> > cells;
-  Result result(cells, true, false, false);
-  EXPECT_EQ(true, result.IsEmpty());
-  EXPECT_EQ(0, result.Size());
-}
-
-TEST(Result, FilledResult) {
-  std::vector<std::shared_ptr<Cell> > cells;
-  PopulateCells(cells);
-
-  Result result(cells, true, false, false);
-
-  EXPECT_EQ(false, result.IsEmpty());
-  EXPECT_EQ(16, result.Size());
-
-  // Get Latest Cell for the given family and qualifier.
-  auto latest_cell(result.ColumnLatestCell("family", "column"));
-  // Nothing of the above family/qualifier combo is present so it should be
-  // nullptr
-  ASSERT_FALSE(latest_cell.get());
-
-  // Try to get the latest cell for the given family and qualifier.
-  latest_cell = result.ColumnLatestCell("family-4", "column-4");
-  // Now shouldn't be a nullptr
-  ASSERT_TRUE(latest_cell.get());
-  // And Value must match too
-  EXPECT_EQ("value-4", latest_cell->Value());
-
-  // Value will be nullptr as no such family and qualifier is present
-  ASSERT_FALSE(result.Value("family-4", "qualifier"));
-  // Value will be present as family and qualifier is present
-  ASSERT_TRUE(result.Value("family-4", "column-4") != nullopt);
-  // Value should be present and match.
-  EXPECT_EQ(latest_cell->Value(), (*result.ColumnLatestCell("family-4", "column-4")).Value());
-  EXPECT_EQ("value-5", (*result.ColumnLatestCell("family-5", "column-5")).Value());
-  EXPECT_EQ("value-8", (*result.ColumnLatestCell("family-8", "column-8")).Value());
-  EXPECT_EQ("value-7", *result.Value("family-7", "column-7"));
-
-  // Get cells for the given family and qualifier
-  auto column_cells = result.ColumnCells("family", "column");
-  // Size should be 0
-  EXPECT_EQ(0, column_cells.size());
-
-  // Size shouldn't be 0 and Row() and Value() must match
-  column_cells = result.ColumnCells("family-0", "column-0");
-  EXPECT_EQ(1, column_cells.size());
-  EXPECT_EQ("row", column_cells[0]->Row());
-  EXPECT_EQ("row", result.Row());
-
-  // Size shouldn't be 0 and Row() and Value() must match
-  column_cells = result.ColumnCells("family-5", "column-5");
-  EXPECT_EQ(2, column_cells.size());
-  EXPECT_EQ("row", column_cells[0]->Row());
-  EXPECT_EQ("row", column_cells[1]->Row());
-  EXPECT_EQ("value-5", column_cells[0]->Value());
-  EXPECT_EQ("value-X", column_cells[1]->Value());
-  EXPECT_EQ("row", result.Row());
-
-  // Size shouldn't be 0 and Row() and Value() must match
-  column_cells = result.ColumnCells("family-8", "column-8");
-  EXPECT_EQ(3, column_cells.size());
-  EXPECT_EQ("row", column_cells[0]->Row());
-  EXPECT_EQ("row", column_cells[1]->Row());
-  EXPECT_EQ("row", column_cells[2]->Row());
-  EXPECT_EQ("value-8", column_cells[0]->Value());
-  EXPECT_EQ("value-X", column_cells[1]->Value());
-  EXPECT_EQ("value-Y", column_cells[2]->Value());
-  EXPECT_EQ("row", result.Row());
-
-  // Size shouldn't be 0 and Row() and Value() must match
-  column_cells = result.ColumnCells("family-9", "column-9");
-  EXPECT_EQ(4, column_cells.size());
-  EXPECT_EQ("row", column_cells[0]->Row());
-  EXPECT_EQ("row", column_cells[1]->Row());
-  EXPECT_EQ("row", column_cells[2]->Row());
-  EXPECT_EQ("row", column_cells[3]->Row());
-  EXPECT_EQ("value-9", column_cells[0]->Value());
-  EXPECT_EQ("value-X", column_cells[1]->Value());
-  EXPECT_EQ("value-Y", column_cells[2]->Value());
-  EXPECT_EQ("value-Z", column_cells[3]->Value());
-  EXPECT_EQ("row", result.Row());
-
-  // Test all the Cell values
-  const auto &result_cells = result.Cells();
-  int i = 0, j = 0;
-  for (const auto &cell : result_cells) {
-    std::string row = "row";
-    std::string family = "family-" + std::to_string(i);
-    std::string column = "column-" + std::to_string(i);
-    std::string value = "value-" + std::to_string(i);
-    switch (j) {
-      case 6:
-      case 10:
-      case 13: {
-        EXPECT_EQ("value-X", cell->Value());
-        ++j;
-        continue;
-      }
-      case 11:
-      case 14: {
-        EXPECT_EQ("value-Y", cell->Value());
-        ++j;
-        continue;
-      }
-      case 15: {
-        EXPECT_EQ("value-Z", cell->Value());
-        ++j;
-        continue;
-      }
-    }
-    EXPECT_EQ(row, cell->Row());
-    EXPECT_EQ(family, cell->Family());
-    EXPECT_EQ(column, cell->Qualifier());
-    EXPECT_EQ(value, cell->Value());
-    ++i;
-    ++j;
-  }
-
-  auto result_map_tmp = result.Map();
-  result_map_tmp["testf"]["testq"][1] = "value";
-  EXPECT_EQ(11, result_map_tmp.size());
-
-  auto result_map = result.Map();
-  EXPECT_EQ(10, result_map.size());
-
-  i = 0;
-  for (auto family_map : result_map) {
-    std::string family = "family-" + std::to_string(i);
-    std::string qualifier = "column-" + std::to_string(i);
-    std::string value = "value-" + std::to_string(i);
-    EXPECT_EQ(family, family_map.first);
-    for (auto qualifier_map : family_map.second) {
-      EXPECT_EQ(qualifier, qualifier_map.first);
-      j = 0;
-      for (auto version_map : qualifier_map.second) {
-        switch (i) {
-          case 5: {
-            if (1 == j) {
-              EXPECT_EQ(1482111803856, version_map.first);
-              EXPECT_EQ("value-X", version_map.second);
-            } else if (0 == j) {
-              EXPECT_EQ(1482113040506, version_map.first);
-              EXPECT_EQ("value-5", version_map.second);
-            }
-            break;
-          }
-          case 8: {
-            if (2 == j) {
-              EXPECT_EQ(1482110969958, version_map.first);
-              EXPECT_EQ("value-Y", version_map.second);
-            } else if (1 == j) {
-              EXPECT_EQ(1482111803856, version_map.first);
-              EXPECT_EQ("value-X", version_map.second);
-            } else if (0 == j) {
-              EXPECT_EQ(1482113040506, version_map.first);
-              EXPECT_EQ("value-8", version_map.second);
-            }
-            break;
-          }
-          case 9: {
-            if (3 == j) {
-              EXPECT_EQ(1482110876075, version_map.first);
-              EXPECT_EQ("value-Z", version_map.second);
-            } else if (2 == j) {
-              EXPECT_EQ(1482110969958, version_map.first);
-              EXPECT_EQ("value-Y", version_map.second);
-            } else if (1 == j) {
-              EXPECT_EQ(1482111803856, version_map.first);
-              EXPECT_EQ("value-X", version_map.second);
-            } else if (0 == j) {
-              EXPECT_EQ(1482113040506, version_map.first);
-              EXPECT_EQ("value-9", version_map.second);
-            }
-            break;
-          }
-          default: {
-            EXPECT_EQ(std::numeric_limits<int64_t>::max(), version_map.first);
-            EXPECT_EQ(value, version_map.second);
-          }
-        }
-        ++j;
-      }
-    }
-    ++i;
-  }
-
-  auto family_map = result.FamilyMap("family-0");
-  EXPECT_EQ(1, family_map.size());
-  i = 0;
-  for (auto qual_val_map : family_map) {
-    EXPECT_EQ("column-0", qual_val_map.first);
-    EXPECT_EQ("value-0", qual_val_map.second);
-  }
-
-  family_map = result.FamilyMap("family-1");
-  EXPECT_EQ(1, family_map.size());
-  i = 0;
-  for (auto qual_val_map : family_map) {
-    EXPECT_EQ("column-1", qual_val_map.first);
-    EXPECT_EQ("value-1", qual_val_map.second);
-  }
-
-  family_map = result.FamilyMap("family-5");
-  EXPECT_EQ(1, family_map.size());
-  i = 0;
-  for (auto qual_val_map : family_map) {
-    EXPECT_EQ("column-5", qual_val_map.first);
-    EXPECT_EQ("value-5", qual_val_map.second);
-  }
-
-  family_map = result.FamilyMap("family-9");
-  EXPECT_EQ(1, family_map.size());
-  i = 0;
-  for (auto qual_val_map : family_map) {
-    EXPECT_EQ("column-9", qual_val_map.first);
-    EXPECT_EQ("value-9", qual_val_map.second);
-  }
-}
-
-TEST(Result, ResultEstimatedSize) {
-  CellType cell_type = CellType::PUT;
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  std::vector<std::shared_ptr<Cell> > cells;
-  Result empty(cells, true, false, false);
-
-  EXPECT_EQ(empty.EstimatedSize(), sizeof(Result));
-
-  cells.push_back(std::make_shared<Cell>("a", "a", "", timestamp, "", cell_type));
-  Result result1(cells, true, false, false);
-  EXPECT_TRUE(result1.EstimatedSize() > empty.EstimatedSize());
-
-  cells.push_back(std::make_shared<Cell>("a", "a", "", timestamp, "", cell_type));
-  Result result2(cells, true, false, false);
-  EXPECT_TRUE(result2.EstimatedSize() > result1.EstimatedSize());
-
-  LOG(INFO) << empty.EstimatedSize();
-  LOG(INFO) << result1.EstimatedSize();
-  LOG(INFO) << result2.EstimatedSize();
-}


[15/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Comparator.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Comparator.proto b/hbase-native-client/if/Comparator.proto
deleted file mode 100644
index 496b68d..0000000
--- a/hbase-native-client/if/Comparator.proto
+++ /dev/null
@@ -1,74 +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.
- */
-
-// This file contains protocol buffers that are used for filters
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ComparatorProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-// This file contains protocol buffers that are used for comparators (e.g. in filters)
-
-message Comparator {
-  required string name = 1;
-  optional bytes serialized_comparator = 2;
-}
-
-message ByteArrayComparable {
-  optional bytes value = 1;
-}
-
-message BinaryComparator {
-  required ByteArrayComparable comparable = 1;
-}
-
-message LongComparator {
-  required ByteArrayComparable comparable = 1;
-}
-
-message BinaryPrefixComparator {
-  required ByteArrayComparable comparable = 1;
-}
-
-message BitComparator {
-  required ByteArrayComparable comparable = 1;
-  required BitwiseOp bitwise_op = 2;
-
-  enum BitwiseOp {
-    AND = 1;
-    OR = 2;
-    XOR = 3;
-  }
-}
-
-message NullComparator {
-}
-
-message RegexStringComparator {
-  required string pattern = 1;
-  required int32 pattern_flags = 2;
-  required string charset = 3;
-  optional string engine = 4;
-}
-
-message SubstringComparator {
-  required string substr = 1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Encryption.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Encryption.proto b/hbase-native-client/if/Encryption.proto
deleted file mode 100644
index 97ab5b2..0000000
--- a/hbase-native-client/if/Encryption.proto
+++ /dev/null
@@ -1,33 +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.
- */
-
-// This file contains protocol buffers used for encryption
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "EncryptionProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message WrappedKey {
-  required string algorithm = 1;
-  required uint32 length = 2;
-  required bytes data = 3;
-  optional bytes iv = 4;
-  optional bytes hash = 5;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/ErrorHandling.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/ErrorHandling.proto b/hbase-native-client/if/ErrorHandling.proto
deleted file mode 100644
index be9a743..0000000
--- a/hbase-native-client/if/ErrorHandling.proto
+++ /dev/null
@@ -1,58 +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.
- */
-
-// This file contains protocol buffers that are used for error handling
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ErrorHandlingProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * Protobuf version of a java.lang.StackTraceElement
- * so we can serialize exceptions.
- */
-message StackTraceElementMessage {
-  optional string declaring_class = 1;
-  optional string method_name = 2;
-  optional string file_name = 3;
-  optional int32 line_number = 4;
-}
-
-/**
- * Cause of a remote failure for a generic exception. Contains
- * all the information for a generic exception as well as
- * optional info about the error for generic info passing
- * (which should be another protobuffed class).
- */
-message GenericExceptionMessage {
-  optional string class_name = 1;
-  optional string message = 2;
-  optional bytes error_info = 3;
-  repeated StackTraceElementMessage trace = 4;
-}
-
-/**
- * Exception sent across the wire when a remote task needs
- * to notify other tasks that it failed and why
- */
-message ForeignExceptionMessage {
-  optional string source = 1;
-  optional GenericExceptionMessage generic_exception = 2;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/FS.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/FS.proto b/hbase-native-client/if/FS.proto
deleted file mode 100644
index 9e93120..0000000
--- a/hbase-native-client/if/FS.proto
+++ /dev/null
@@ -1,45 +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.
- */
-
-// This file contains protocol buffers that are written into the filesystem
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "FSProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-/**
- * The ${HBASE_ROOTDIR}/hbase.version file content
- */
-message HBaseVersionFileContent {
-  required string version = 1;
-}
-
-/**
- * Reference file content used when we split an hfile under a region.
- */
-message Reference {
-  required bytes splitkey = 1;
-  enum Range {
-    TOP = 0;
-    BOTTOM = 1;
-  }
-  required Range range = 2;
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Filter.proto b/hbase-native-client/if/Filter.proto
deleted file mode 100644
index 1fa6697..0000000
--- a/hbase-native-client/if/Filter.proto
+++ /dev/null
@@ -1,171 +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.
- */
-
-// This file contains protocol buffers that are used for filters
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "FilterProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Comparator.proto";
-
-message Filter {
-  required string name = 1;
-  optional bytes serialized_filter = 2;
-}
-
-message ColumnCountGetFilter {
-  required int32 limit = 1;
-}
-
-message ColumnPaginationFilter {
-  required int32 limit = 1;
-  optional int32 offset = 2;
-  optional bytes column_offset = 3;
-}
-
-message ColumnPrefixFilter {
-  required bytes prefix = 1;
-}
-
-message ColumnRangeFilter {
-  optional bytes min_column = 1;
-  optional bool min_column_inclusive = 2;
-  optional bytes max_column = 3;
-  optional bool max_column_inclusive = 4;
-}
-
-message CompareFilter {
-  required CompareType compare_op = 1;
-  optional Comparator comparator = 2;
-}
-
-message DependentColumnFilter {
-  required CompareFilter compare_filter = 1;
-  optional bytes column_family = 2;
-  optional bytes column_qualifier = 3;
-  optional bool drop_dependent_column = 4;
-}
-
-message FamilyFilter {
-  required CompareFilter compare_filter = 1;
-}
-
-message FilterList {
-  required Operator operator = 1;
-  repeated Filter filters = 2;
-
-  enum Operator {
-    MUST_PASS_ALL = 1;
-    MUST_PASS_ONE = 2;
-  }
-}
-
-message FilterWrapper {
-  required Filter filter = 1;
-}
-
-message FirstKeyOnlyFilter {
-}
-
-message FirstKeyValueMatchingQualifiersFilter {
-  repeated bytes qualifiers = 1;
-}
-
-message FuzzyRowFilter {
-  repeated BytesBytesPair fuzzy_keys_data = 1;
-}
-
-message InclusiveStopFilter {
-  optional bytes stop_row_key = 1;
-}
-
-message KeyOnlyFilter {
-  required bool len_as_val = 1;
-}
-
-message MultipleColumnPrefixFilter {
-  repeated bytes sorted_prefixes = 1;
-}
-
-message PageFilter {
-  required int64 page_size = 1;
-}
-
-message PrefixFilter {
-  optional bytes prefix = 1;
-}
-
-message QualifierFilter {
-  required CompareFilter compare_filter = 1;
-}
-
-message RandomRowFilter {
-  required float chance = 1;
-}
-
-message RowFilter {
-  required CompareFilter compare_filter = 1;
-}
-
-message SingleColumnValueExcludeFilter {
-  required SingleColumnValueFilter single_column_value_filter = 1;
-}
-
-message SingleColumnValueFilter {
-  optional bytes column_family = 1;
-  optional bytes column_qualifier = 2;
-  required CompareType compare_op = 3;
-  required Comparator comparator = 4;
-  optional bool filter_if_missing = 5;
-  optional bool latest_version_only = 6;
-}
-
-message SkipFilter {
-  required Filter filter = 1;
-}
-
-message TimestampsFilter {
-  repeated int64 timestamps = 1 [packed=true];
-  optional bool can_hint = 2;
-}
-
-message ValueFilter {
-  required CompareFilter compare_filter = 1;
-}
-
-message WhileMatchFilter {
-  required Filter filter = 1;
-}
-message FilterAllFilter {
-}
-
-message RowRange {
-  optional bytes start_row = 1;
-  optional bool start_row_inclusive = 2;
-  optional bytes stop_row = 3;
-  optional bool stop_row_inclusive =4;
-}
-
-message MultiRowRangeFilter {
-  repeated RowRange row_range_list = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/HBase.proto b/hbase-native-client/if/HBase.proto
deleted file mode 100644
index c36b214..0000000
--- a/hbase-native-client/if/HBase.proto
+++ /dev/null
@@ -1,253 +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.
- */
-
-// This file contains protocol buffers that are shared throughout HBase
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "HBaseProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "Cell.proto";
-
-/**
- * Table Name
- */
-message TableName {
-  required bytes namespace = 1;
-  required bytes qualifier = 2;
-}
-
-/**
- * Table Schema
- * Inspired by the rest TableSchema
- */
-message TableSchema {
-  optional TableName table_name = 1;
-  repeated BytesBytesPair attributes = 2;
-  repeated ColumnFamilySchema column_families = 3;
-  repeated NameStringPair configuration = 4;
-}
-
-/** Denotes state of the table */
-message TableState {
-  // Table's current state
-  enum State {
-    ENABLED = 0;
-    DISABLED = 1;
-    DISABLING = 2;
-    ENABLING = 3;
-  }
-  // This is the table's state.
-  required State state = 1;
-}
-
-/**
- * Column Family Schema
- * Inspired by the rest ColumSchemaMessage
- */
-message ColumnFamilySchema {
-  required bytes name = 1;
-  repeated BytesBytesPair attributes = 2;
-  repeated NameStringPair configuration = 3;
-}
-
-/**
- * Protocol buffer version of HRegionInfo.
- */
-message RegionInfo {
-  required uint64 region_id = 1;
-  required TableName table_name = 2;
-  optional bytes start_key = 3;
-  optional bytes end_key = 4;
-  optional bool offline = 5;
-  optional bool split = 6;
-  optional int32 replica_id = 7 [default = 0];
-}
-
-/**
- * Protocol buffer for favored nodes
- */
-message FavoredNodes {
-  repeated ServerName favored_node = 1;
-}
-
-/**
- * Container protocol buffer to specify a region.
- * You can specify region by region name, or the hash
- * of the region name, which is known as encoded
- * region name.
- */
-message RegionSpecifier {
-  required RegionSpecifierType type = 1;
-  required bytes value = 2;
-
-  enum RegionSpecifierType {
-    // <tablename>,<startkey>,<regionId>.<encodedName>
-    REGION_NAME = 1;
-
-    // hash of <tablename>,<startkey>,<regionId>
-    ENCODED_REGION_NAME = 2;
-  }
-}
-
-/**
- * A range of time. Both from and to are Java time
- * stamp in milliseconds. If you don't specify a time
- * range, it means all time.  By default, if not
- * specified, from = 0, and to = Long.MAX_VALUE
- */
-message TimeRange {
-  optional uint64 from = 1;
-  optional uint64 to = 2;
-}
-
-/* ColumnFamily Specific TimeRange */
-message ColumnFamilyTimeRange {
-  required bytes column_family = 1;
-  required TimeRange time_range = 2;
-}
-
-/* Comparison operators */
-enum CompareType {
-  LESS = 0;
-  LESS_OR_EQUAL = 1;
-  EQUAL = 2;
-  NOT_EQUAL = 3;
-  GREATER_OR_EQUAL = 4;
-  GREATER = 5;
-  NO_OP = 6;
-}
-
-/**
- * Protocol buffer version of ServerName
- */
-message ServerName {
-  required string host_name = 1;
-  optional uint32 port = 2;
-  optional uint64 start_code = 3;
-}
-
-// Comment data structures
-
-message Coprocessor {
-  required string name = 1;
-}
-
-message NameStringPair {
-  required string name = 1;
-  required string value = 2;
-}
-
-message NameBytesPair {
-  required string name = 1;
-  optional bytes value = 2;
-}
-
-message BytesBytesPair {
-  required bytes first = 1;
-  required bytes second = 2;
-}
-
-message NameInt64Pair {
-  optional string name = 1;
-  optional int64 value = 2;
-}
-
-/**
- * Description of the snapshot to take
- */
-message SnapshotDescription {
-  required string name = 1;
-  optional string table = 2; // not needed for delete, but checked for in taking snapshot
-  optional int64 creation_time = 3 [default = 0];
-  enum Type {
-    DISABLED = 0;
-    FLUSH = 1;
-    SKIPFLUSH = 2;
-  }
-  optional Type type = 4 [default = FLUSH];
-  optional int32 version = 5;
-  optional string owner = 6;
-}
-
-/**
- * Description of the distributed procedure to take
- */
-message ProcedureDescription {
-  required string signature = 1; // the unique signature of the procedure
-  optional string instance = 2; // the procedure instance name
-  optional int64 creation_time = 3 [default = 0];
-  repeated NameStringPair configuration = 4;
-}
-
-message EmptyMsg {
-}
-
-enum TimeUnit {
-  NANOSECONDS = 1;
-  MICROSECONDS = 2;
-  MILLISECONDS = 3;
-  SECONDS = 4;
-  MINUTES = 5;
-  HOURS = 6;
-  DAYS = 7;
-}
-
-message LongMsg {
-  required int64 long_msg = 1;
-}
-
-message DoubleMsg {
-  required double double_msg = 1;
-}
-
-message BigDecimalMsg {
-  required bytes bigdecimal_msg = 1;
-}
-
-message UUID {
-  required uint64 least_sig_bits = 1;
-  required uint64 most_sig_bits = 2;
-}
-
-message NamespaceDescriptor {
-  required bytes name = 1;
-  repeated NameStringPair configuration = 2;
-}
-
-// Rpc client version info proto. Included in ConnectionHeader on connection setup
-message VersionInfo {
-  required string version = 1;
-  required string url = 2;
-  required string revision = 3;
-  required string user = 4;
-  required string date = 5;
-  required string src_checksum = 6;
-  optional uint32 version_major = 7;
-  optional uint32 version_minor = 8;
-}
-
-/**
- * Description of the region server info
- */
-message RegionServerInfo {
-  optional int32 infoPort = 1;
-  optional VersionInfo version_info = 2;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/HFile.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/HFile.proto b/hbase-native-client/if/HFile.proto
deleted file mode 100644
index 5c5e4f3..0000000
--- a/hbase-native-client/if/HFile.proto
+++ /dev/null
@@ -1,49 +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.
- */
-
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "HFileProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-// Map of name/values
-message FileInfoProto {
-  repeated BytesBytesPair map_entry = 1;
-}
-
-// HFile file trailer
-message FileTrailerProto {
-  optional uint64 file_info_offset = 1;
-  optional uint64 load_on_open_data_offset = 2;
-  optional uint64 uncompressed_data_index_size = 3;
-  optional uint64 total_uncompressed_bytes = 4;
-  optional uint32 data_index_count = 5;
-  optional uint32 meta_index_count = 6;
-  optional uint64 entry_count = 7;
-  optional uint32 num_data_index_levels = 8;
-  optional uint64 first_data_block_offset = 9;
-  optional uint64 last_data_block_offset = 10;
-  optional string comparator_class_name = 11;
-  optional uint32 compression_codec = 12;
-  optional bytes encryption_key = 13;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/LoadBalancer.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/LoadBalancer.proto b/hbase-native-client/if/LoadBalancer.proto
deleted file mode 100644
index f9c5d0d..0000000
--- a/hbase-native-client/if/LoadBalancer.proto
+++ /dev/null
@@ -1,29 +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.
- */
-
-// This file contains protocol buffers to represent the state of the load balancer.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "LoadBalancerProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message LoadBalancerState {
-  optional bool balancer_on = 1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/MapReduce.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/MapReduce.proto b/hbase-native-client/if/MapReduce.proto
deleted file mode 100644
index f96ffdf..0000000
--- a/hbase-native-client/if/MapReduce.proto
+++ /dev/null
@@ -1,37 +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.
- */
-
- //This file includes protocol buffers used in MapReduce only.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "MapReduceProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message ScanMetrics {
-  repeated NameInt64Pair metrics = 1;
-}
-
-message TableSnapshotRegionSplit {
-  repeated string locations = 2;
-  optional TableSchema table = 3;
-  optional RegionInfo region = 4;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Master.proto b/hbase-native-client/if/Master.proto
deleted file mode 100644
index ad8111e..0000000
--- a/hbase-native-client/if/Master.proto
+++ /dev/null
@@ -1,828 +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.
- */
-
-// All to do with the Master.  Includes schema management since these
-// changes are run by the Master process.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "MasterProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Client.proto";
-import "ClusterStatus.proto";
-import "ErrorHandling.proto";
-import "Procedure.proto";
-import "Quota.proto";
-
-/* Column-level protobufs */
-
-message AddColumnRequest {
-  required TableName table_name = 1;
-  required ColumnFamilySchema column_families = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message AddColumnResponse {
-  optional uint64 proc_id = 1;
-}
-
-message DeleteColumnRequest {
-  required TableName table_name = 1;
-  required bytes column_name = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message DeleteColumnResponse {
-  optional uint64 proc_id = 1;
-}
-
-message ModifyColumnRequest {
-  required TableName table_name = 1;
-  required ColumnFamilySchema column_families = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message ModifyColumnResponse {
-  optional uint64 proc_id = 1;
-}
-
-/* Region-level Protos */
-
-message MoveRegionRequest {
-  required RegionSpecifier region = 1;
-  optional ServerName dest_server_name = 2;
-}
-
-message MoveRegionResponse {
-}
-
-/**
- * Dispatch merging the specified regions.
- */
-message DispatchMergingRegionsRequest {
-  required RegionSpecifier region_a = 1;
-  required RegionSpecifier region_b = 2;
-  optional bool forcible = 3 [default = false];
-}
-
-message DispatchMergingRegionsResponse {
-}
-
-message AssignRegionRequest {
-  required RegionSpecifier region = 1;
-}
-
-message AssignRegionResponse {
-}
-
-message UnassignRegionRequest {
-  required RegionSpecifier region = 1;
-  optional bool force = 2 [default = false];
-}
-
-message UnassignRegionResponse {
-}
-
-message OfflineRegionRequest {
-  required RegionSpecifier region = 1;
-}
-
-message OfflineRegionResponse {
-}
-
-/* Table-level protobufs */
-
-message CreateTableRequest {
-  required TableSchema table_schema = 1;
-  repeated bytes split_keys = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message CreateTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-message DeleteTableRequest {
-  required TableName table_name = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message DeleteTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-message TruncateTableRequest {
-  required TableName tableName = 1;
-  optional bool preserveSplits = 2 [default = false];
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message TruncateTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-message EnableTableRequest {
-  required TableName table_name = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message EnableTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-message DisableTableRequest {
-  required TableName table_name = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message DisableTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-message ModifyTableRequest {
-  required TableName table_name = 1;
-  required TableSchema table_schema = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message ModifyTableResponse {
-  optional uint64 proc_id = 1;
-}
-
-/* Namespace-level protobufs */
-
-message CreateNamespaceRequest {
-  required NamespaceDescriptor namespaceDescriptor = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message CreateNamespaceResponse {
-  optional uint64 proc_id = 1;
-}
-
-message DeleteNamespaceRequest {
-  required string namespaceName = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message DeleteNamespaceResponse {
-  optional uint64 proc_id = 1;
-}
-
-message ModifyNamespaceRequest {
-  required NamespaceDescriptor namespaceDescriptor = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message ModifyNamespaceResponse {
-  optional uint64 proc_id = 1;
-}
-
-message GetNamespaceDescriptorRequest {
-  required string namespaceName = 1;
-}
-
-message GetNamespaceDescriptorResponse {
-  required NamespaceDescriptor namespaceDescriptor = 1;
-}
-
-message ListNamespaceDescriptorsRequest {
-}
-
-message ListNamespaceDescriptorsResponse {
-  repeated NamespaceDescriptor namespaceDescriptor = 1;
-}
-
-message ListTableDescriptorsByNamespaceRequest {
-  required string namespaceName = 1;
-}
-
-message ListTableDescriptorsByNamespaceResponse {
-  repeated TableSchema tableSchema = 1;
-}
-
-message ListTableNamesByNamespaceRequest {
-  required string namespaceName = 1;
-}
-
-message ListTableNamesByNamespaceResponse {
-  repeated TableName tableName = 1;
-}
-
-/* Cluster-level protobufs */
-
-
-message ShutdownRequest {
-}
-
-message ShutdownResponse {
-}
-
-message StopMasterRequest {
-}
-
-message StopMasterResponse {
-}
-
-message BalanceRequest {
-  optional bool force = 1;
-}
-
-message BalanceResponse {
-  required bool balancer_ran = 1;
-}
-
-message SetBalancerRunningRequest {
-  required bool on = 1;
-  optional bool synchronous = 2;
-}
-
-message SetBalancerRunningResponse {
-  optional bool prev_balance_value = 1;
-}
-
-message IsBalancerEnabledRequest {
-}
-
-message IsBalancerEnabledResponse {
-  required bool enabled = 1;
-}
-
-enum MasterSwitchType {
-  SPLIT = 0;
-  MERGE = 1;
-}
-
-message SetSplitOrMergeEnabledRequest {
-  required bool enabled = 1;
-  optional bool synchronous = 2;
-  repeated MasterSwitchType switch_types = 3;
-  optional bool skip_lock = 4;
-}
-
-message SetSplitOrMergeEnabledResponse {
-  repeated bool prev_value = 1;
-}
-
-message IsSplitOrMergeEnabledRequest {
-  required MasterSwitchType switch_type = 1;
-}
-
-message IsSplitOrMergeEnabledResponse {
-  required bool enabled = 1;
-}
-
-message ReleaseSplitOrMergeLockAndRollbackRequest {
-}
-
-message ReleaseSplitOrMergeLockAndRollbackResponse {
-}
-
-message NormalizeRequest {
-}
-
-message NormalizeResponse {
-  required bool normalizer_ran = 1;
-}
-
-message SetNormalizerRunningRequest {
-  required bool on = 1;
-}
-
-message SetNormalizerRunningResponse {
-  optional bool prev_normalizer_value = 1;
-}
-
-message IsNormalizerEnabledRequest {
-}
-
-message IsNormalizerEnabledResponse {
-  required bool enabled = 1;
-}
-
-message RunCatalogScanRequest {
-}
-
-message RunCatalogScanResponse {
-  optional int32 scan_result = 1;
-}
-
-message EnableCatalogJanitorRequest {
-  required bool enable = 1;
-}
-
-message EnableCatalogJanitorResponse {
-  optional bool prev_value = 1;
-}
-
-message IsCatalogJanitorEnabledRequest {
-}
-
-message IsCatalogJanitorEnabledResponse {
-  required bool value = 1;
-}
-
-message SnapshotRequest {
-	required SnapshotDescription snapshot = 1;
-}
-
-message SnapshotResponse {
-	required int64 expected_timeout = 1;
-}
-
-message GetCompletedSnapshotsRequest {
-}
-
-message GetCompletedSnapshotsResponse {
-	repeated SnapshotDescription snapshots = 1;
-}
-
-message DeleteSnapshotRequest {
-	required SnapshotDescription snapshot = 1;
-}
-
-message DeleteSnapshotResponse {
-}
-
-message RestoreSnapshotRequest {
-  required SnapshotDescription snapshot = 1;
-  optional uint64 nonce_group = 2 [default = 0];
-  optional uint64 nonce = 3 [default = 0];
-}
-
-message RestoreSnapshotResponse {
-  required uint64 proc_id = 1;
-}
-
-/* if you don't send the snapshot, then you will get it back
- * in the response (if the snapshot is done) so you can check the snapshot
- */
-message IsSnapshotDoneRequest {
-	optional SnapshotDescription snapshot = 1;
-}
-
-message IsSnapshotDoneResponse {
-	optional bool done = 1 [default = false];
-	optional SnapshotDescription snapshot = 2;
-}
-
-message IsRestoreSnapshotDoneRequest {
-  optional SnapshotDescription snapshot = 1;
-}
-
-message IsRestoreSnapshotDoneResponse {
-  optional bool done = 1 [default = false];
-}
-
-message GetSchemaAlterStatusRequest {
-  required TableName table_name = 1;
-}
-
-message GetSchemaAlterStatusResponse {
-  optional uint32 yet_to_update_regions = 1;
-  optional uint32 total_regions = 2;
-}
-
-message GetTableDescriptorsRequest {
-  repeated TableName table_names = 1;
-  optional string regex = 2;
-  optional bool include_sys_tables = 3 [default=false];
-  optional string namespace = 4;
-}
-
-message GetTableDescriptorsResponse {
-  repeated TableSchema table_schema = 1;
-}
-
-message GetTableNamesRequest {
-  optional string regex = 1;
-  optional bool include_sys_tables = 2 [default=false];
-  optional string namespace = 3;
-}
-
-message GetTableNamesResponse {
-  repeated TableName table_names = 1;
-}
-
-message GetTableStateRequest {
-  required TableName table_name = 1;
-}
-
-message GetTableStateResponse {
-  required TableState table_state = 1;
-}
-
-message GetClusterStatusRequest {
-}
-
-message GetClusterStatusResponse {
-  required ClusterStatus cluster_status = 1;
-}
-
-message IsMasterRunningRequest {
-}
-
-message IsMasterRunningResponse {
-  required bool is_master_running = 1;
-}
-
-message ExecProcedureRequest {
-  required ProcedureDescription procedure = 1;
-}
-
-message ExecProcedureResponse {
-  optional int64 expected_timeout = 1;
-  optional bytes return_data = 2;
-}
-
-message IsProcedureDoneRequest {
-  optional ProcedureDescription procedure = 1;
-}
-
-message IsProcedureDoneResponse {
-  optional bool done = 1 [default = false];
-  optional ProcedureDescription snapshot = 2;
-}
-
-message GetProcedureResultRequest {
-  required uint64 proc_id = 1;
-}
-
-message GetProcedureResultResponse {
-  enum State {
-    NOT_FOUND = 0;
-    RUNNING = 1;
-    FINISHED = 2;
-  }
-
-  required State state = 1;
-  optional uint64 start_time = 2;
-  optional uint64 last_update = 3;
-  optional bytes result = 4;
-  optional ForeignExceptionMessage exception = 5;
-}
-
-message AbortProcedureRequest {
-  required uint64 proc_id = 1;
-  optional bool mayInterruptIfRunning = 2 [default = true];
-}
-
-message AbortProcedureResponse {
-  required bool is_procedure_aborted = 1;
-}
-
-message ListProceduresRequest {
-}
-
-message ListProceduresResponse {
-  repeated Procedure procedure = 1;
-}
-
-message SetQuotaRequest {
-  optional string user_name = 1;
-  optional string user_group = 2;
-  optional string namespace = 3;
-  optional TableName table_name = 4;
-
-  optional bool remove_all = 5;
-  optional bool bypass_globals = 6;
-  optional ThrottleRequest throttle = 7;
-}
-
-message SetQuotaResponse {
-}
-
-message MajorCompactionTimestampRequest {
-  required TableName table_name = 1;
-}
-
-message MajorCompactionTimestampForRegionRequest {
-  required RegionSpecifier region = 1;
-}
-
-message MajorCompactionTimestampResponse {
-  required int64 compaction_timestamp = 1;
-}
-
-message SecurityCapabilitiesRequest {
-}
-
-message SecurityCapabilitiesResponse {
-  enum Capability {
-    SIMPLE_AUTHENTICATION = 0;
-    SECURE_AUTHENTICATION = 1;
-    AUTHORIZATION = 2;
-    CELL_AUTHORIZATION = 3;
-    CELL_VISIBILITY = 4;
-  }
-
-  repeated Capability capabilities = 1;
-}
-
-service MasterService {
-  /** Used by the client to get the number of regions that have received the updated schema */
-  rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
-    returns(GetSchemaAlterStatusResponse);
-
-  /** Get list of TableDescriptors for requested tables. */
-  rpc GetTableDescriptors(GetTableDescriptorsRequest)
-    returns(GetTableDescriptorsResponse);
-
-  /** Get the list of table names. */
-  rpc GetTableNames(GetTableNamesRequest)
-    returns(GetTableNamesResponse);
-
-  /** Return cluster status. */
-  rpc GetClusterStatus(GetClusterStatusRequest)
-    returns(GetClusterStatusResponse);
-
-  /** return true if master is available */
-  rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse);
-
-  /** Adds a column to the specified table. */
-  rpc AddColumn(AddColumnRequest)
-    returns(AddColumnResponse);
-
-  /** Deletes a column from the specified table. Table must be disabled. */
-  rpc DeleteColumn(DeleteColumnRequest)
-    returns(DeleteColumnResponse);
-
-  /** Modifies an existing column on the specified table. */
-  rpc ModifyColumn(ModifyColumnRequest)
-    returns(ModifyColumnResponse);
-
-  /** Move the region region to the destination server. */
-  rpc MoveRegion(MoveRegionRequest)
-    returns(MoveRegionResponse);
-
- /** Master dispatch merging the regions */
-  rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
-    returns(DispatchMergingRegionsResponse);
-
-  /** Assign a region to a server chosen at random. */
-  rpc AssignRegion(AssignRegionRequest)
-    returns(AssignRegionResponse);
-
-  /**
-   * Unassign a region from current hosting regionserver.  Region will then be
-   * assigned to a regionserver chosen at random.  Region could be reassigned
-   * back to the same server.  Use MoveRegion if you want
-   * to control the region movement.
-   */
-  rpc UnassignRegion(UnassignRegionRequest)
-    returns(UnassignRegionResponse);
-
-  /**
-   * Offline a region from the assignment manager's in-memory state.  The
-   * region should be in a closed state and there will be no attempt to
-   * automatically reassign the region as in unassign.   This is a special
-   * method, and should only be used by experts or hbck.
-   */
-  rpc OfflineRegion(OfflineRegionRequest)
-    returns(OfflineRegionResponse);
-
-  /** Deletes a table */
-  rpc DeleteTable(DeleteTableRequest)
-    returns(DeleteTableResponse);
-
-  /** Truncate a table */
-  rpc truncateTable(TruncateTableRequest)
-    returns(TruncateTableResponse);
-
-  /** Puts the table on-line (only needed if table has been previously taken offline) */
-  rpc EnableTable(EnableTableRequest)
-    returns(EnableTableResponse);
-
-  /** Take table offline */
-  rpc DisableTable(DisableTableRequest)
-    returns(DisableTableResponse);
-
-  /** Modify a table's metadata */
-  rpc ModifyTable(ModifyTableRequest)
-    returns(ModifyTableResponse);
-
-  /** Creates a new table asynchronously */
-  rpc CreateTable(CreateTableRequest)
-    returns(CreateTableResponse);
-
-    /** Shutdown an HBase cluster. */
-  rpc Shutdown(ShutdownRequest)
-    returns(ShutdownResponse);
-
-  /** Stop HBase Master only.  Does not shutdown the cluster. */
-  rpc StopMaster(StopMasterRequest)
-    returns(StopMasterResponse);
-
-  /**
-   * Run the balancer.  Will run the balancer and if regions to move, it will
-   * go ahead and do the reassignments.  Can NOT run for various reasons.
-   * Check logs.
-   */
-  rpc Balance(BalanceRequest)
-    returns(BalanceResponse);
-
-  /**
-   * Turn the load balancer on or off.
-   * If synchronous is true, it waits until current balance() call, if outstanding, to return.
-   */
-  rpc SetBalancerRunning(SetBalancerRunningRequest)
-    returns(SetBalancerRunningResponse);
-
-  /**
-   * Query whether the Region Balancer is running.
-   */
-  rpc IsBalancerEnabled(IsBalancerEnabledRequest)
-    returns(IsBalancerEnabledResponse);
-
-  /**
-   * Turn the split or merge switch on or off.
-   * If synchronous is true, it waits until current operation call, if outstanding, to return.
-   */
-  rpc SetSplitOrMergeEnabled(SetSplitOrMergeEnabledRequest)
-    returns(SetSplitOrMergeEnabledResponse);
-
-  /**
-   * Query whether the split or merge switch is on/off.
-   */
-  rpc IsSplitOrMergeEnabled(IsSplitOrMergeEnabledRequest)
-    returns(IsSplitOrMergeEnabledResponse);
-
-  /**
-   * Release lock and rollback state.
-   */
-  rpc ReleaseSplitOrMergeLockAndRollback(ReleaseSplitOrMergeLockAndRollbackRequest)
-    returns(ReleaseSplitOrMergeLockAndRollbackResponse);
-
-  /**
-   * Run region normalizer. Can NOT run for various reasons. Check logs.
-   */
-  rpc Normalize(NormalizeRequest)
-    returns(NormalizeResponse);
-
-  /**
-   * Turn region normalizer on or off.
-   */
-  rpc SetNormalizerRunning(SetNormalizerRunningRequest)
-    returns(SetNormalizerRunningResponse);
-
-  /**
-   * Query whether region normalizer is enabled.
-   */
-  rpc IsNormalizerEnabled(IsNormalizerEnabledRequest)
-    returns(IsNormalizerEnabledResponse);
-
-  /** Get a run of the catalog janitor */
-  rpc RunCatalogScan(RunCatalogScanRequest)
-     returns(RunCatalogScanResponse);
-
-  /**
-   * Enable the catalog janitor on or off.
-   */
-  rpc EnableCatalogJanitor(EnableCatalogJanitorRequest)
-     returns(EnableCatalogJanitorResponse);
-
-  /**
-   * Query whether the catalog janitor is enabled.
-   */
-  rpc IsCatalogJanitorEnabled(IsCatalogJanitorEnabledRequest)
-     returns(IsCatalogJanitorEnabledResponse);
-
-  /**
-   * Call a master coprocessor endpoint
-   */
-  rpc ExecMasterService(CoprocessorServiceRequest)
-    returns(CoprocessorServiceResponse);
-
-  /**
-   * Create a snapshot for the given table.
-   */
-  rpc Snapshot(SnapshotRequest) returns(SnapshotResponse);
-
-  /**
-   * Get completed snapshots.
-   * Returns a list of snapshot descriptors for completed snapshots
-   */
-  rpc GetCompletedSnapshots(GetCompletedSnapshotsRequest) returns(GetCompletedSnapshotsResponse);
-
-  /**
-   * Delete an existing snapshot. This method can also be used to clean up an aborted snapshot.
-   */
-  rpc DeleteSnapshot(DeleteSnapshotRequest) returns(DeleteSnapshotResponse);
-
-  /**
-   * Determine if the snapshot is done yet.
-   */
-  rpc IsSnapshotDone(IsSnapshotDoneRequest) returns(IsSnapshotDoneResponse);
-
-  /**
-   * Restore a snapshot
-   */
-  rpc RestoreSnapshot(RestoreSnapshotRequest) returns(RestoreSnapshotResponse);
-
-  /**
-   * Execute a distributed procedure.
-   */
-  rpc ExecProcedure(ExecProcedureRequest) returns(ExecProcedureResponse);
-
-  /**
-   * Execute a distributed procedure with return data.
-   */
-  rpc ExecProcedureWithRet(ExecProcedureRequest) returns(ExecProcedureResponse);
-
-  /**
-   * Determine if the procedure is done yet.
-   */
-  rpc IsProcedureDone(IsProcedureDoneRequest) returns(IsProcedureDoneResponse);
-
-  /** return true if master is available */
-  /** rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse); */
-
-  /** Modify a namespace's metadata */
-  rpc ModifyNamespace(ModifyNamespaceRequest)
-    returns(ModifyNamespaceResponse);
-
-  /** Creates a new namespace synchronously */
-  rpc CreateNamespace(CreateNamespaceRequest)
-    returns(CreateNamespaceResponse);
-
-  /** Deletes namespace synchronously */
-  rpc DeleteNamespace(DeleteNamespaceRequest)
-    returns(DeleteNamespaceResponse);
-
-  /** Get a namespace descriptor by name */
-  rpc GetNamespaceDescriptor(GetNamespaceDescriptorRequest)
-    returns(GetNamespaceDescriptorResponse);
-
-  /** returns a list of namespaces */
-  rpc ListNamespaceDescriptors(ListNamespaceDescriptorsRequest)
-    returns(ListNamespaceDescriptorsResponse);
-
-  /** returns a list of tables for a given namespace*/
-  rpc ListTableDescriptorsByNamespace(ListTableDescriptorsByNamespaceRequest)
-    returns(ListTableDescriptorsByNamespaceResponse);
-
-  /** returns a list of tables for a given namespace*/
-  rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
-    returns(ListTableNamesByNamespaceResponse);
-
-  /** returns table state */
-  rpc GetTableState(GetTableStateRequest)
-    returns(GetTableStateResponse);
-
-  /** Apply the new quota settings */
-  rpc SetQuota(SetQuotaRequest) returns(SetQuotaResponse);
-
-  /** Returns the timestamp of the last major compaction */
-  rpc getLastMajorCompactionTimestamp(MajorCompactionTimestampRequest)
-    returns(MajorCompactionTimestampResponse);
-
-  /** Returns the timestamp of the last major compaction */
-  rpc getLastMajorCompactionTimestampForRegion(MajorCompactionTimestampForRegionRequest)
-    returns(MajorCompactionTimestampResponse);
-
-  rpc getProcedureResult(GetProcedureResultRequest)
-    returns(GetProcedureResultResponse);
-
-  /** Returns the security capabilities in effect on the cluster */
-  rpc getSecurityCapabilities(SecurityCapabilitiesRequest)
-    returns(SecurityCapabilitiesResponse);
-
-  /** Abort a procedure */
-  rpc AbortProcedure(AbortProcedureRequest)
-    returns(AbortProcedureResponse);
-
-  /** returns a list of procedures */
-  rpc ListProcedures(ListProceduresRequest)
-    returns(ListProceduresResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/MasterProcedure.proto b/hbase-native-client/if/MasterProcedure.proto
deleted file mode 100644
index 87aae6a..0000000
--- a/hbase-native-client/if/MasterProcedure.proto
+++ /dev/null
@@ -1,285 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "MasterProcedureProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "RPC.proto";
-
-// ============================================================================
-//  WARNING - Compatibility rules
-// ============================================================================
-// This .proto contains the data serialized by the master procedures.
-// Each procedure has some state stored to know, which step were executed
-// and what were the parameters or data created by the previous steps.
-// new code should be able to handle the old format or at least fail cleanly
-// triggering a rollback/cleanup.
-//
-// Procedures that are inheriting from a StateMachineProcedure have an enum:
-//  - Do not change the number of the 'State' enums.
-//    doing so, will cause executing the wrong 'step' on the pending
-//    procedures when they will be replayed.
-//  - Do not remove items from the enum, new code must be able to handle
-//    all the previous 'steps'. There may be pending procedure ready to be
-//    recovered replayed. alternative you can make sure that not-known state
-//    will result in a failure that will rollback the already executed steps.
-// ============================================================================
-
-enum CreateTableState {
-  CREATE_TABLE_PRE_OPERATION = 1;
-  CREATE_TABLE_WRITE_FS_LAYOUT = 2;
-  CREATE_TABLE_ADD_TO_META = 3;
-  CREATE_TABLE_ASSIGN_REGIONS = 4;
-  CREATE_TABLE_UPDATE_DESC_CACHE = 5;
-  CREATE_TABLE_POST_OPERATION = 6;
-}
-
-message CreateTableStateData {
-  required UserInformation user_info = 1;
-  required TableSchema table_schema = 2;
-  repeated RegionInfo region_info = 3;
-}
-
-enum ModifyTableState {
-  MODIFY_TABLE_PREPARE = 1;
-  MODIFY_TABLE_PRE_OPERATION = 2;
-  MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;
-  MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;
-  MODIFY_TABLE_DELETE_FS_LAYOUT = 5;
-  MODIFY_TABLE_POST_OPERATION = 6;
-  MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;
-}
-
-message ModifyTableStateData {
-  required UserInformation user_info = 1;
-  optional TableSchema unmodified_table_schema = 2;
-  required TableSchema modified_table_schema = 3;
-  required bool delete_column_family_in_modify = 4;
-}
-
-enum TruncateTableState {
-  TRUNCATE_TABLE_PRE_OPERATION = 1;
-  TRUNCATE_TABLE_REMOVE_FROM_META = 2;
-  TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;
-  TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;
-  TRUNCATE_TABLE_ADD_TO_META = 5;
-  TRUNCATE_TABLE_ASSIGN_REGIONS = 6;
-  TRUNCATE_TABLE_POST_OPERATION = 7;
-}
-
-message TruncateTableStateData {
-  required UserInformation user_info = 1;
-  required bool preserve_splits = 2;
-  optional TableName table_name = 3;
-  optional TableSchema table_schema = 4;
-  repeated RegionInfo region_info = 5;
-}
-
-enum DeleteTableState {
-  DELETE_TABLE_PRE_OPERATION = 1;
-  DELETE_TABLE_REMOVE_FROM_META = 2;
-  DELETE_TABLE_CLEAR_FS_LAYOUT = 3;
-  DELETE_TABLE_UPDATE_DESC_CACHE = 4;
-  DELETE_TABLE_UNASSIGN_REGIONS = 5;
-  DELETE_TABLE_POST_OPERATION = 6;
-}
-
-message DeleteTableStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  repeated RegionInfo region_info = 3;
-}
-
-enum CreateNamespaceState {
-  CREATE_NAMESPACE_PREPARE = 1;
-  CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
-  CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
-  CREATE_NAMESPACE_UPDATE_ZK = 4;
-  CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
-}
-
-message CreateNamespaceStateData {
-  required NamespaceDescriptor namespace_descriptor = 1;
-}
-
-enum ModifyNamespaceState {
-  MODIFY_NAMESPACE_PREPARE = 1;
-  MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
-  MODIFY_NAMESPACE_UPDATE_ZK = 3;
-}
-
-message ModifyNamespaceStateData {
-  required NamespaceDescriptor namespace_descriptor = 1;
-  optional NamespaceDescriptor unmodified_namespace_descriptor = 2;
-}
-
-enum DeleteNamespaceState {
-  DELETE_NAMESPACE_PREPARE = 1;
-  DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
-  DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
-  DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
-  DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
-}
-
-message DeleteNamespaceStateData {
-  required string namespace_name = 1;
-  optional NamespaceDescriptor namespace_descriptor = 2;
-}
-
-enum AddColumnFamilyState {
-  ADD_COLUMN_FAMILY_PREPARE = 1;
-  ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
-  ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
-  ADD_COLUMN_FAMILY_POST_OPERATION = 4;
-  ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
-}
-
-message AddColumnFamilyStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  required ColumnFamilySchema columnfamily_schema = 3;
-  optional TableSchema unmodified_table_schema = 4;
-}
-
-enum ModifyColumnFamilyState {
-  MODIFY_COLUMN_FAMILY_PREPARE = 1;
-  MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
-  MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
-  MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
-  MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
-}
-
-message ModifyColumnFamilyStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  required ColumnFamilySchema columnfamily_schema = 3;
-  optional TableSchema unmodified_table_schema = 4;
-}
-
-enum DeleteColumnFamilyState {
-  DELETE_COLUMN_FAMILY_PREPARE = 1;
-  DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
-  DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
-  DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
-  DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
-  DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
-}
-
-message DeleteColumnFamilyStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  required bytes columnfamily_name = 3;
-  optional TableSchema unmodified_table_schema = 4;
-}
-
-enum EnableTableState {
-  ENABLE_TABLE_PREPARE = 1;
-  ENABLE_TABLE_PRE_OPERATION = 2;
-  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
-  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
-  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
-  ENABLE_TABLE_POST_OPERATION = 6;
-}
-
-message EnableTableStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  required bool skip_table_state_check = 3;
-}
-
-enum DisableTableState {
-  DISABLE_TABLE_PREPARE = 1;
-  DISABLE_TABLE_PRE_OPERATION = 2;
-  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
-  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
-  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
-  DISABLE_TABLE_POST_OPERATION = 6;
-}
-
-message DisableTableStateData {
-  required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  required bool skip_table_state_check = 3;
-}
-
-message RestoreParentToChildRegionsPair {
-  required string parent_region_name = 1;
-  required string child1_region_name = 2;
-  required string child2_region_name = 3;
-}
-
-enum CloneSnapshotState {
-  CLONE_SNAPSHOT_PRE_OPERATION = 1;
-  CLONE_SNAPSHOT_WRITE_FS_LAYOUT = 2;
-  CLONE_SNAPSHOT_ADD_TO_META = 3;
-  CLONE_SNAPSHOT_ASSIGN_REGIONS = 4;
-  CLONE_SNAPSHOT_UPDATE_DESC_CACHE = 5;
-  CLONE_SNAPSHOT_POST_OPERATION = 6;
-}
-
-message CloneSnapshotStateData {
-  required UserInformation user_info = 1;
-  required SnapshotDescription snapshot = 2;
-  required TableSchema table_schema = 3;
-  repeated RegionInfo region_info = 4;
-  repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5;
-}
-
-enum RestoreSnapshotState {
-  RESTORE_SNAPSHOT_PRE_OPERATION = 1;
-  RESTORE_SNAPSHOT_UPDATE_TABLE_DESCRIPTOR = 2;
-  RESTORE_SNAPSHOT_WRITE_FS_LAYOUT = 3;
-  RESTORE_SNAPSHOT_UPDATE_META = 4;
-}
-
-message RestoreSnapshotStateData {
-  required UserInformation user_info = 1;
-  required SnapshotDescription snapshot = 2;
-  required TableSchema modified_table_schema = 3;
-  repeated RegionInfo region_info_for_restore = 4;
-  repeated RegionInfo region_info_for_remove = 5;
-  repeated RegionInfo region_info_for_add = 6;
-  repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
-}
-
-message ServerCrashStateData {
-  required ServerName server_name = 1;
-  optional bool distributed_log_replay = 2;
-  repeated RegionInfo regions_on_crashed_server = 3;
-  repeated RegionInfo regions_assigned = 4;
-  optional bool carrying_meta = 5;
-  optional bool should_split_wal = 6 [default = true];
-}
-
-enum ServerCrashState {
-  SERVER_CRASH_START = 1;
-  SERVER_CRASH_PROCESS_META = 2;
-  SERVER_CRASH_GET_REGIONS = 3;
-  SERVER_CRASH_NO_SPLIT_LOGS = 4;
-  SERVER_CRASH_SPLIT_LOGS = 5;
-  SERVER_CRASH_PREPARE_LOG_REPLAY = 6;
-  // Removed SERVER_CRASH_CALC_REGIONS_TO_ASSIGN = 7;
-  SERVER_CRASH_ASSIGN = 8;
-  SERVER_CRASH_WAIT_ON_ASSIGN = 9;
-  SERVER_CRASH_FINISH = 100;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/MultiRowMutation.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/MultiRowMutation.proto b/hbase-native-client/if/MultiRowMutation.proto
deleted file mode 100644
index 747afac..0000000
--- a/hbase-native-client/if/MultiRowMutation.proto
+++ /dev/null
@@ -1,45 +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.
- */
-package hbase.pb;
-
-import "Client.proto";
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "MultiRowMutationProtos";
-option java_generate_equals_and_hash = true;
-option java_generic_services = true;
-option optimize_for = SPEED;
-
-message MultiRowMutationProcessorRequest{
-}
-
-message MultiRowMutationProcessorResponse{
-}
-
-message MutateRowsRequest {
-  repeated MutationProto mutation_request = 1;
-  optional uint64 nonce_group = 2;
-  optional uint64 nonce = 3;
-}
-
-message MutateRowsResponse {
-}
-
-service MultiRowMutationService {
-  rpc MutateRows(MutateRowsRequest)
-      returns(MutateRowsResponse);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Procedure.proto b/hbase-native-client/if/Procedure.proto
deleted file mode 100644
index 55e44a4..0000000
--- a/hbase-native-client/if/Procedure.proto
+++ /dev/null
@@ -1,119 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ProcedureProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "ErrorHandling.proto";
-
-enum ProcedureState {
-  INITIALIZING = 1;         // Procedure in construction, not yet added to the executor
-  RUNNABLE = 2;             // Procedure added to the executor, and ready to be executed
-  WAITING = 3;              // The procedure is waiting on children to be completed
-  WAITING_TIMEOUT = 4;      // The procedure is waiting a timout or an external event
-  ROLLEDBACK = 5;           // The procedure failed and was rolledback
-  FINISHED = 6;             // The procedure execution is completed. may need a rollback if failed.
-}
-
-/**
- * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
- */
-message Procedure {
-  // internal "static" state
-  required string class_name = 1;        // full classname to be able to instantiate the procedure
-  optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
-  required uint64 proc_id = 3;
-  required uint64 start_time = 4;
-  optional string owner = 5;
-
-  // internal "runtime" state
-  required ProcedureState state = 6;
-  repeated uint32 stack_id = 7;          // stack indices in case the procedure was running
-  required uint64 last_update = 8;
-  optional uint32 timeout = 9;
-
-  // user state/results
-  optional ForeignExceptionMessage exception = 10;
-  optional bytes result = 11;           // opaque (user) result structure
-  optional bytes state_data = 12;       // opaque (user) procedure internal-state
-
-  // Nonce to prevent same procedure submit by multiple times
-  optional uint64 nonce_group = 13 [default = 0];
-  optional uint64 nonce = 14 [default = 0];
-}
-
-/**
- * SequentialProcedure data
- */
-message SequentialProcedureData {
-  required bool executed = 1;
-}
-
-/**
- * StateMachineProcedure data
- */
-message StateMachineProcedureData {
-  repeated uint32 state = 1;
-}
-
-/**
- * Procedure WAL header
- */
-message ProcedureWALHeader {
-  required uint32 version = 1;
-  required uint32 type = 2;
-  required uint64 log_id = 3;
-  required uint64 min_proc_id = 4;
-}
-
-/**
- * Procedure WAL trailer
- */
-message ProcedureWALTrailer {
-  required uint32 version = 1;
-  required uint64 tracker_pos = 2;
-}
-
-message ProcedureStoreTracker {
-  message TrackerNode {
-    required uint64 start_id = 1;
-    repeated uint64 updated = 2;
-    repeated uint64 deleted = 3;
-  }
-
-  repeated TrackerNode node = 1;
-}
-
-message ProcedureWALEntry {
-  enum Type {
-    PROCEDURE_WAL_EOF     = 1;
-    PROCEDURE_WAL_INIT    = 2;
-    PROCEDURE_WAL_INSERT  = 3;
-    PROCEDURE_WAL_UPDATE  = 4;
-    PROCEDURE_WAL_DELETE  = 5;
-    PROCEDURE_WAL_COMPACT = 6;
-  }
-
-  required Type type = 1;
-  repeated Procedure procedure = 2;
-  optional uint64 proc_id = 3;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Quota.proto b/hbase-native-client/if/Quota.proto
deleted file mode 100644
index a8303b1..0000000
--- a/hbase-native-client/if/Quota.proto
+++ /dev/null
@@ -1,76 +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.
- */
-
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "QuotaProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-enum QuotaScope {
-  CLUSTER = 1;
-  MACHINE = 2;
-}
-
-message TimedQuota {
-  required TimeUnit time_unit = 1;
-  optional uint64 soft_limit  = 2;
-  optional float share = 3;
-  optional QuotaScope scope  = 4 [default = MACHINE];
-}
-
-enum ThrottleType {
-  REQUEST_NUMBER = 1;
-  REQUEST_SIZE   = 2;
-  WRITE_NUMBER   = 3;
-  WRITE_SIZE     = 4;
-  READ_NUMBER    = 5;
-  READ_SIZE      = 6;
-}
-
-message Throttle {
-  optional TimedQuota req_num  = 1;
-  optional TimedQuota req_size = 2;
-
-  optional TimedQuota write_num  = 3;
-  optional TimedQuota write_size = 4;
-
-  optional TimedQuota read_num  = 5;
-  optional TimedQuota read_size = 6;
-}
-
-message ThrottleRequest {
-  optional ThrottleType type = 1;
-  optional TimedQuota timed_quota = 2;
-}
-
-enum QuotaType {
-  THROTTLE = 1;
-}
-
-message Quotas {
-  optional bool bypass_globals = 1 [default = false];
-  optional Throttle throttle = 2;
-}
-
-message QuotaUsage {
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RPC.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RPC.proto b/hbase-native-client/if/RPC.proto
deleted file mode 100644
index 8413d25..0000000
--- a/hbase-native-client/if/RPC.proto
+++ /dev/null
@@ -1,137 +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.
- */
-package hbase.pb;
-
-import "Tracing.proto"; 
-import "HBase.proto";
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RPCProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-// See https://issues.apache.org/jira/browse/HBASE-7898 for high-level
-// description of RPC specification.
-//
-// On connection setup, the client sends six bytes of preamble -- a four
-// byte magic, a byte of version, and a byte of authentication type.
-//
-// We then send a "ConnectionHeader" protobuf of user information and the
-// 'protocol' or 'service' that is to be run over this connection as well as
-// info such as codecs and compression to use when we send cell blocks(see below).
-// This connection header protobuf is prefaced by an int that holds the length
-// of this connection header (this is NOT a varint).  The pb connection header
-// is sent with Message#writeTo.  The server throws an exception if it doesn't
-// like what it was sent noting what it is objecting too.  Otherwise, the server
-// says nothing and is open for business.
-//
-// Hereafter the client makes requests and the server returns responses.
-//
-// Requests look like this:
-//
-// <An int with the total length of the request>
-// <RequestHeader Message written out using Message#writeDelimitedTo>
-// <Optionally a Request Parameter Message written out using Message#writeDelimitedTo>
-// <Optionally a Cell block>
-//
-// ...where the Request Parameter Message is whatever the method name stipulated
-// in the RequestHeader expects; e.g. if the method is a scan, then the pb
-// Request Message is a GetRequest, or a ScanRequest.  A block of Cells
-// optionally follows.  The presence of a Request param Message and/or a
-// block of Cells will be noted in the RequestHeader.
-//
-// Response is the mirror of the request:
-//
-// <An int with the total length of the response>
-// <ResponseHeader Message written out using Message#writeDelimitedTo>
-// <Optionally a Response Result Message written out using Message#writeDelimitedTo>
-// <Optionally a Cell block>
-//
-// ...where the Response Message is the response type that goes with the
-// method specified when making the request and the follow on Cell blocks may
-// or may not be there -- read the response header to find out if one following.
-// If an exception, it will be included inside the Response Header.
-//
-// Any time we write a pb, we do it with Message#writeDelimitedTo EXCEPT when
-// the connection header is sent; this is prefaced by an int with its length
-// and the pb connection header is then written with Message#writeTo.
-//
-
-// User Information proto.  Included in ConnectionHeader on connection setup
-message UserInformation {
-  required string effective_user = 1;
-  optional string real_user = 2;
-}
-
-// This is sent on connection setup after the connection preamble is sent.
-message ConnectionHeader {
-  optional UserInformation user_info = 1;
-  optional string service_name = 2;
-  // Cell block codec we will use sending over optional cell blocks.  Server throws exception
-  // if cannot deal.  Null means no codec'ing going on so we are pb all the time (SLOW!!!)
-  optional string cell_block_codec_class = 3;
-  // Compressor we will use if cell block is compressed.  Server will throw exception if not supported.
-  // Class must implement hadoop's CompressionCodec Interface.  Can't compress if no codec.
-  optional string cell_block_compressor_class = 4;
-  optional VersionInfo version_info = 5;
-}
-
-// Optional Cell block Message.  Included in client RequestHeader
-message CellBlockMeta {
-  // Length of the following cell block.  Could calculate it but convenient having it too hand.
-  optional uint32 length = 1;
-}
-
-// At the RPC layer, this message is used to carry
-// the server side exception to the RPC client.
-message ExceptionResponse {
-  // Class name of the exception thrown from the server
-  optional string exception_class_name = 1;
-  // Exception stack trace from the server side
-  optional string stack_trace = 2;
-  // Optional hostname.  Filled in for some exceptions such as region moved
-  // where exception gives clue on where the region may have moved.
-  optional string hostname = 3;
-  optional int32 port = 4;
-  // Set if we are NOT to retry on receipt of this exception
-  optional bool do_not_retry = 5;
-}
-
-// Header sent making a request.
-message RequestHeader {
-  // Monotonically increasing call_id to keep track of RPC requests and their response
-  optional uint32 call_id = 1;
-  optional RPCTInfo trace_info = 2;
-  optional string method_name = 3;
-  // If true, then a pb Message param follows.
-  optional bool request_param = 4;
-  // If present, then an encoded data block follows.
-  optional CellBlockMeta cell_block_meta = 5;
-  // 0 is NORMAL priority.  200 is HIGH.  If no priority, treat it as NORMAL.
-  // See HConstants.
-  optional uint32 priority = 6;
-  optional uint32 timeout = 7;
-}
-
-message ResponseHeader {
-  optional uint32 call_id = 1;
-  // If present, then request threw an exception and no response message (else we presume one)
-  optional ExceptionResponse exception = 2;
-  // If present, then an encoded data block follows.
-  optional CellBlockMeta cell_block_meta = 3;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RSGroup.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RSGroup.proto b/hbase-native-client/if/RSGroup.proto
deleted file mode 100644
index 7358941..0000000
--- a/hbase-native-client/if/RSGroup.proto
+++ /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.
- */
-
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RSGroupProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-
-message RSGroupInfo {
-  required string name = 1;
-  repeated ServerName servers = 4;
-  repeated TableName tables = 3;
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RSGroupAdmin.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RSGroupAdmin.proto b/hbase-native-client/if/RSGroupAdmin.proto
deleted file mode 100644
index fda9b09..0000000
--- a/hbase-native-client/if/RSGroupAdmin.proto
+++ /dev/null
@@ -1,136 +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.
- */
-
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RSGroupAdminProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "RSGroup.proto";
-
-/** Group level protobufs */
-
-message ListTablesOfRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message ListTablesOfRSGroupResponse {
-  repeated TableName table_name = 1;
-}
-
-message GetRSGroupInfoRequest {
-  required string r_s_group_name = 1;
-}
-
-message GetRSGroupInfoResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfTableRequest {
-  required TableName table_name = 1;
-}
-
-message GetRSGroupInfoOfTableResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-message MoveServersRequest {
-  required string target_group = 1;
-  repeated ServerName servers = 3;
-}
-
-message MoveServersResponse {
-}
-
-message MoveTablesRequest {
-  required string target_group = 1;
-  repeated TableName table_name = 2;
-}
-
-message MoveTablesResponse {
-}
-
-message AddRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message AddRSGroupResponse {
-}
-
-message RemoveRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message RemoveRSGroupResponse {
-}
-
-message BalanceRSGroupRequest {
-  required string r_s_group_name = 1;
-}
-
-message BalanceRSGroupResponse {
-  required bool balanceRan = 1;
-}
-
-message ListRSGroupInfosRequest {
-}
-
-message ListRSGroupInfosResponse {
-  repeated RSGroupInfo  r_s_group_info = 1;
-}
-
-message GetRSGroupInfoOfServerRequest {
-  required ServerName server = 2;
-}
-
-message GetRSGroupInfoOfServerResponse {
-  optional RSGroupInfo r_s_group_info = 1;
-}
-
-service RSGroupAdminService {
-  rpc GetRSGroupInfo(GetRSGroupInfoRequest)
-    returns (GetRSGroupInfoResponse);
-
-  rpc GetRSGroupInfoOfTable(GetRSGroupInfoOfTableRequest)
-    returns (GetRSGroupInfoOfTableResponse);
-
-  rpc GetRSGroupInfoOfServer(GetRSGroupInfoOfServerRequest)
-    returns (GetRSGroupInfoOfServerResponse);
-
-  rpc MoveServers(MoveServersRequest)
-    returns (MoveServersResponse);
-
-  rpc MoveTables(MoveTablesRequest)
-    returns (MoveTablesResponse);
-
-  rpc AddRSGroup(AddRSGroupRequest)
-    returns (AddRSGroupResponse);
-
-  rpc RemoveRSGroup(RemoveRSGroupRequest)
-    returns (RemoveRSGroupResponse);
-
-  rpc BalanceRSGroup(BalanceRSGroupRequest)
-    returns (BalanceRSGroupResponse);
-
-  rpc ListRSGroupInfos(ListRSGroupInfosRequest)
-    returns (ListRSGroupInfosResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RegionNormalizer.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RegionNormalizer.proto b/hbase-native-client/if/RegionNormalizer.proto
deleted file mode 100644
index e5305d6..0000000
--- a/hbase-native-client/if/RegionNormalizer.proto
+++ /dev/null
@@ -1,28 +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.
- */
-
-// This file contains protocol buffers to represent the state of the load balancer.
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RegionNormalizerProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message RegionNormalizerState {
-    optional bool normalizer_on = 1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RegionServerStatus.proto b/hbase-native-client/if/RegionServerStatus.proto
deleted file mode 100644
index fda9de2..0000000
--- a/hbase-native-client/if/RegionServerStatus.proto
+++ /dev/null
@@ -1,158 +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.
- */
-
-// This file contains protocol buffers that are used for RegionServerStatusProtocol.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RegionServerStatusProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "ClusterStatus.proto";
-
-message RegionServerStartupRequest {
-  /** Port number this regionserver is up on */
-  required uint32 port = 1;
-
-  /** This servers' startcode */
-  required uint64 server_start_code = 2;
-
-  /** Current time of the region server in ms */
-  required uint64 server_current_time = 3;
-
-  /** hostname for region server, optional */
-  optional string use_this_hostname_instead = 4;
-}
-
-message RegionServerStartupResponse {
-  /**
-   * Configuration for the regionserver to use: e.g. filesystem,
-   * hbase rootdir, the hostname to use creating the RegionServer ServerName,
-   * etc
-   */
-  repeated NameStringPair map_entries = 1;
-}
-
-message RegionServerReportRequest {
-  required ServerName server = 1;
-
-  /** load the server is under */
-  optional ServerLoad load = 2;
-}
-
-message RegionServerReportResponse {
-}
-
-message ReportRSFatalErrorRequest {
-  /** name of the server experiencing the error */
-  required ServerName server = 1;
-
-  /** informative text to expose in the master logs and UI */
-  required string error_message = 2;
-}
-
-message ReportRSFatalErrorResponse {
-}
-
-message GetLastFlushedSequenceIdRequest {
-  /** region name */
-  required bytes region_name = 1;
-}
-
-message GetLastFlushedSequenceIdResponse {
-  /** the last WAL sequence id flushed from MemStore to HFile for the region */
-  required uint64 last_flushed_sequence_id = 1;
-
-  /** the last WAL sequence id flushed from MemStore to HFile for stores of the region */
-  repeated StoreSequenceId store_last_flushed_sequence_id = 2;
-}
-
-message RegionStateTransition {
-  required TransitionCode transition_code = 1;
-
-  /** Mutliple regions are involved during merging/splitting */
-  repeated RegionInfo region_info = 2;
-
-  /** For newly opened region, the open seq num is needed */
-  optional uint64 open_seq_num = 3;
-
-  enum TransitionCode {
-    OPENED = 0;
-    FAILED_OPEN = 1;
-    /** No failed_close, in which case region server will abort */
-    CLOSED = 2;
-
-    /** Ask master for ok to split/merge region(s) */
-    READY_TO_SPLIT = 3;
-    READY_TO_MERGE = 4;
-
-    SPLIT_PONR = 5;
-    MERGE_PONR = 6;
-
-    SPLIT = 7;
-    MERGED = 8;
-    SPLIT_REVERTED = 9;
-    MERGE_REVERTED = 10;
-  }
-}
-
-message ReportRegionStateTransitionRequest {
-  /** This region server's server name */
-  required ServerName server = 1;
-
-  repeated RegionStateTransition transition = 2;
-}
-
-message ReportRegionStateTransitionResponse {
-  /** Error message if failed to update the region state */
-  optional string error_message = 1;
-}
-
-service RegionServerStatusService {
-  /** Called when a region server first starts. */
-  rpc RegionServerStartup(RegionServerStartupRequest)
-    returns(RegionServerStartupResponse);
-
-  /** Called to report the load the RegionServer is under. */
-  rpc RegionServerReport(RegionServerReportRequest)
-    returns(RegionServerReportResponse);
-
-  /**
-   * Called by a region server to report a fatal error that is causing it to
-   * abort.
-   */
-  rpc ReportRSFatalError(ReportRSFatalErrorRequest)
-    returns(ReportRSFatalErrorResponse);
-
-  /** Called to get the sequence id of the last MemStore entry flushed to an
-   * HFile for a specified region. Used by the region server to speed up
-   * log splitting. */
-  rpc GetLastFlushedSequenceId(GetLastFlushedSequenceIdRequest)
-    returns(GetLastFlushedSequenceIdResponse);
-
-  /**
-   * Called by a region server to report the progress of a region
-   * transition. If the request fails, the transition should
-   * be aborted.
-   */
-  rpc ReportRegionStateTransition(ReportRegionStateTransitionRequest)
-    returns(ReportRegionStateTransitionResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/RowProcessor.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/RowProcessor.proto b/hbase-native-client/if/RowProcessor.proto
deleted file mode 100644
index cf2f30f..0000000
--- a/hbase-native-client/if/RowProcessor.proto
+++ /dev/null
@@ -1,45 +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.
- */
-/**
- * Defines a protocol to perform multi row transactions.
- * See BaseRowProcessorEndpoint for the implementation.
- * See HRegion#processRowsWithLocks() for details.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "RowProcessorProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-message ProcessRequest {
-  required string row_processor_class_name = 1;
-  optional string row_processor_initializer_message_name = 2;
-  optional bytes  row_processor_initializer_message = 3;
-  optional uint64 nonce_group = 4;
-  optional uint64 nonce = 5;
-}
-
-message ProcessResponse {
-  required bytes row_processor_result = 1;
-}
-
-service RowProcessorService {
-  rpc Process(ProcessRequest) returns (ProcessResponse);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/SecureBulkLoad.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/SecureBulkLoad.proto b/hbase-native-client/if/SecureBulkLoad.proto
deleted file mode 100644
index 814735b..0000000
--- a/hbase-native-client/if/SecureBulkLoad.proto
+++ /dev/null
@@ -1,72 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "SecureBulkLoadProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import 'HBase.proto';
-import 'Client.proto';
-
-message SecureBulkLoadHFilesRequest {
-  repeated BulkLoadHFileRequest.FamilyPath family_path = 1;
-  optional bool assign_seq_num = 2;
-  required DelegationToken fs_token = 3;
-  required string bulk_token = 4;
-}
-
-message SecureBulkLoadHFilesResponse {
-  required bool loaded = 1;
-}
-
-message DelegationToken {
-  optional bytes identifier = 1;
-  optional bytes password = 2;
-  optional string kind = 3;
-  optional string service = 4;
-}
-
-message PrepareBulkLoadRequest {
-  required TableName table_name = 1;
-}
-
-message PrepareBulkLoadResponse {
-  required string bulk_token = 1;
-}
-
-message CleanupBulkLoadRequest {
-  required string bulk_token = 1;
-
-}
-
-message CleanupBulkLoadResponse {
-}
-
-service SecureBulkLoadService {
-    rpc PrepareBulkLoad(PrepareBulkLoadRequest)
-      returns (PrepareBulkLoadResponse);
-
-    rpc SecureBulkLoadHFiles(SecureBulkLoadHFilesRequest)
-      returns (SecureBulkLoadHFilesResponse);
-
-    rpc CleanupBulkLoad(CleanupBulkLoadRequest)
-      returns (CleanupBulkLoadResponse);
-}


[07/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/hbase-configuration-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/hbase-configuration-test.cc b/hbase-native-client/src/hbase/client/hbase-configuration-test.cc
new file mode 100644
index 0000000..173c4df
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/hbase-configuration-test.cc
@@ -0,0 +1,375 @@
+/*
+ * 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 <fstream>
+#include <iostream>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <boost/filesystem.hpp>
+#include "hbase/client/configuration.h"
+#include "hbase/client/hbase-configuration-loader.h"
+
+using namespace hbase;
+using std::experimental::nullopt;
+
+const std::string kDefHBaseConfPath("./build/test-data/hbase-configuration-test/conf/");
+const std::string kHBaseConfPath("./build/test-data/hbase-configuration-test/custom-conf/");
+
+const std::string kHBaseDefaultXml("hbase-default.xml");
+const std::string kHBaseSiteXml("hbase-site.xml");
+
+const std::string kHBaseDefaultXmlData(
+    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
+    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
+    "Software Foundation (ASF) under one\n * or more contributor license "
+    "agreements.  See the NOTICE file\n * distributed with this work for "
+    "additional information\n * regarding copyright ownership.  The ASF "
+    "licenses this file\n * to you under the Apache License, Version 2.0 "
+    "(the\n * \"License\"); you may not use this file except in compliance\n * "
+    "with the License.  You may obtain a copy of the License at\n *\n *     "
+    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
+    "applicable law or agreed to in writing, software\n * distributed under "
+    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
+    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
+    "for the specific language governing permissions and\n * limitations under "
+    "the License.\n "
+    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
+    "name>\n<value>/root/hbase-docker/apps/hbase/data</value>\n<final>true</"
+    "final>\n</"
+    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
+    "name>\n<value>This value will be "
+    "overwritten</value>\n<final>false</final>\n</"
+    "property>\n\n<property>\n<name>default-prop</name>\n<value>default-value</"
+    "value>\n</property>\n\n</configuration>");
+const std::string kHBaseSiteXmlData(
+    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
+    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
+    "Software Foundation (ASF) under one\n * or more contributor license "
+    "agreements.  See the NOTICE file\n * distributed with this work for "
+    "additional information\n * regarding copyright ownership.  The ASF "
+    "licenses this file\n * to you under the Apache License, Version 2.0 "
+    "(the\n * \"License\"); you may not use this file except in compliance\n * "
+    "with the License.  You may obtain a copy of the License at\n *\n *     "
+    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
+    "applicable law or agreed to in writing, software\n * distributed under "
+    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
+    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
+    "for the specific language governing permissions and\n * limitations under "
+    "the License.\n "
+    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
+    "name>\n<value>This value will not be be "
+    "overwritten</value>\n</"
+    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
+    "name>\n<value>/root/hbase-docker/zookeeper</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.name</"
+    "name>\n<value>${user.name}</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.dir</"
+    "name>\n<value>${user.dir}</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.home</"
+    "name>\n<value>${user.home}</value>\n</"
+    "property>\n\n<property>\n<name>selfRef</name>\n<value>${selfRef}</"
+    "value>\n</property>\n\n<property>\n<name>foo.substs</"
+    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
+    "bar},${bar},</value>\n</"
+    "property>\n\n<property>\n<name>foo.substs.exception</"
+    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
+    "bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar}"
+    ",${bar},${bar}</value>\n</property>\n\n<property>\n<name>bar</"
+    "name>\n<value>bar-value</value>\n</"
+    "property>\n\n<property>\n<name>custom-prop</name>\n<value>custom-value</"
+    "value>\n</property>\n\n<property>\n<name>int</name>\n<value>16000</"
+    "value>\n</property>\n\n<property>\n<name>int.largevalue</"
+    "name>\n<value>2147483646</value>\n</"
+    "property>\n\n<property>\n<name>int.exception</name>\n<value>2147483648</"
+    "value>\n</property>\n\n<property>\n<name>long</name>\n<value>2147483850</"
+    "value>\n</property>\n\n<property>\n<name>long.largevalue</"
+    "name>\n<value>9223372036854775807</value>\n</"
+    "property>\n\n<property>\n<name>long.exception</"
+    "name>\n<value>9223372036854775810</value>\n</"
+    "property>\n\n<property>\n<name>double</name>\n<value>17.9769e+100</"
+    "value>\n</property>\n\n<property>\n<name>double.largevalue</"
+    "name>\n<value>170.769e+200</value>\n</"
+    "property>\n\n<property>\n<name>double.exception</"
+    "name>\n<value>1.79769e+310</value>\n</"
+    "property>\n\n<property>\n<name>bool.true</name>\n<value>true</value>\n</"
+    "property>\n\n<property>\n<name>bool.false</name>\n<value>false</value>\n</"
+    "property>\n\n<property>\n<name>bool.exception</name>\n<value>unknown "
+    "bool</value>\n</property>\n\n</configuration>");
+
+void WriteDataToFile(const std::string &file, const std::string &xml_data) {
+  std::ofstream hbase_conf;
+  hbase_conf.open(file.c_str());
+  hbase_conf << xml_data;
+  hbase_conf.close();
+}
+
+void CreateHBaseConf(const std::string &dir, const std::string &file, const std::string xml_data) {
+  // Directory will be created if not present
+  if (!boost::filesystem::exists(dir)) {
+    boost::filesystem::create_directories(dir);
+  }
+  // Remove temp file always
+  boost::filesystem::remove((dir + file).c_str());
+  WriteDataToFile((dir + file), xml_data);
+}
+
+void CreateHBaseConfWithEnv() {
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 1);
+}
+
+/*
+ * Config will be loaded from $HBASE_CONF. We set it @ kDefHBaseConfPath
+ * Config values will be loaded from hbase-default.xml and hbase-site.xml
+ * present in the above path.
+ */
+TEST(Configuration, LoadConfFromDefaultLocation) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 0);
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value").c_str(), "custom-value");
+  EXPECT_STREQ((*conf).Get("default-prop", "Set this value").c_str(), "default-value");
+}
+
+/*
+ * Config will be loaded from hbase-site.xml defined at
+ * kHBaseConfPath
+ */
+TEST(Configuration, LoadConfFromCustomLocation) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+
+  HBaseConfigurationLoader loader;
+  std::vector<std::string> resources{kHBaseSiteXml};
+  hbase::optional<Configuration> conf = loader.LoadResources(kHBaseConfPath, resources);
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
+  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
+}
+
+/*
+ * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
+ * kDefHBaseConfPath and kHBaseConfPath respectively.
+ */
+TEST(Configuration, LoadConfFromMultipleLocatons) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  CreateHBaseConf(kHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+
+  HBaseConfigurationLoader loader;
+  std::string conf_paths = kDefHBaseConfPath + ":" + kHBaseConfPath;
+  std::vector<std::string> resources{kHBaseDefaultXml, kHBaseSiteXml};
+  hbase::optional<Configuration> conf = loader.LoadResources(conf_paths, resources);
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("default-prop", "From hbase-default.xml").c_str(), "default-value");
+  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
+  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
+}
+
+/*
+ * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
+ * $HBASE_CONF.
+ * We set HBASE_CONF to kDefHBaseConfPath
+ * Below tests load the conf files in the same way unless specified.
+ */
+TEST(Configuration, DefaultValues) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("default-prop", "Set this value.").c_str(), "default-value");
+  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value.").c_str(), "custom-value");
+}
+
+TEST(Configuration, FinalValues) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("hbase.rootdir", "").c_str(), "/root/hbase-docker/apps/hbase/data");
+  EXPECT_STREQ((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
+               "/root/hbase-docker/zookeeper");
+  EXPECT_STRNE((*conf).Get("hbase.rootdir", "").c_str(), "This value will not be be overwritten");
+  EXPECT_STRNE((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
+               "This value will be overwritten");
+}
+
+/*
+ * Config will be loaded from HBASE_CONF which we set in
+ * CreateHBaseConfWithEnv().
+ * Config values will be loaded from hbase-default.xml and hbase-site.xml in the
+ * above path.
+ */
+TEST(Configuration, EnvVars) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("hbase-client.user.name", "").c_str(), "${user.name}");
+  EXPECT_STRNE((*conf).Get("hbase-client.user.name", "root").c_str(), "test-user");
+}
+
+TEST(Configuration, SelfRef) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("selfRef", "${selfRef}").c_str(), "${selfRef}");
+}
+
+TEST(Configuration, VarExpansion) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("foo.substs", "foo-value").c_str(),
+               "bar-value,bar-value,bar-value,bar-value,bar-value,bar-value,"
+               "bar-value,bar-value,bar-value,bar-value,");
+  EXPECT_STRNE((*conf).Get("foo.substs", "foo-value").c_str(), "bar-value");
+}
+
+TEST(Configuration, VarExpansionException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  ASSERT_THROW((*conf).Get("foo.substs.exception", "foo-value").c_str(), std::runtime_error);
+}
+
+TEST(Configuration, GetInt) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_EQ(16000, (*conf).GetInt("int", 0));
+  EXPECT_EQ(2147483646, (*conf).GetInt("int.largevalue", 0));
+}
+
+TEST(Configuration, GetLong) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_EQ(2147483850, (*conf).GetLong("long", 0));
+  EXPECT_EQ(9223372036854775807, (*conf).GetLong("long.largevalue", 0));
+}
+
+TEST(Configuration, GetDouble) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_DOUBLE_EQ(17.9769e+100, (*conf).GetDouble("double", 0.0));
+  EXPECT_DOUBLE_EQ(170.769e+200, (*conf).GetDouble("double.largevalue", 0.0));
+}
+
+TEST(Configuration, GetBool) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  EXPECT_EQ(true, (*conf).GetBool("bool.true", true));
+  EXPECT_EQ(false, (*conf).GetBool("bool.false", false));
+}
+
+TEST(Configuration, GetIntException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetInt("int.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetLongException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetLong("long.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetDoubleException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetDouble("double.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetBoolException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetBool("bool.exception", false), std::runtime_error);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/hbase-rpc-controller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/hbase-rpc-controller.cc b/hbase-native-client/src/hbase/client/hbase-rpc-controller.cc
new file mode 100644
index 0000000..c2ae6cc
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/hbase-rpc-controller.cc
@@ -0,0 +1,22 @@
+/*
+ * 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 "hbase/client/hbase-rpc-controller.h"
+
+namespace hbase {} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/increment-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/increment-test.cc b/hbase-native-client/src/hbase/client/increment-test.cc
new file mode 100644
index 0000000..3c11fbe
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/increment-test.cc
@@ -0,0 +1,129 @@
+/*
+ * 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 <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "hbase/client/increment.h"
+#include "hbase/client/mutation.h"
+#include "hbase/client/put.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Increment;
+using hbase::Increment;
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Mutation;
+using hbase::TimeUtil;
+
+const constexpr int64_t Mutation::kLatestTimestamp;
+
+TEST(Increment, Row) {
+  Increment incr{"foo"};
+  EXPECT_EQ("foo", incr.row());
+}
+
+TEST(Increment, Durability) {
+  Increment incr{"row"};
+  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, incr.Durability());
+
+  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
+  incr.SetDurability(skipWal);
+  EXPECT_EQ(skipWal, incr.Durability());
+}
+
+TEST(Increment, Timestamp) {
+  Increment incr{"row"};
+
+  // test default timestamp
+  EXPECT_EQ(Mutation::kLatestTimestamp, incr.TimeStamp());
+
+  // set custom timestamp
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  incr.SetTimeStamp(ts);
+  EXPECT_EQ(ts, incr.TimeStamp());
+
+  // Add a column with custom timestamp
+  incr.AddColumn("f", "q", 5l);
+  auto &cell = incr.FamilyMap().at("f")[0];
+  EXPECT_EQ(ts, cell->Timestamp());
+}
+
+TEST(Increment, HasFamilies) {
+  Increment incr{"row"};
+
+  EXPECT_EQ(false, incr.HasFamilies());
+
+  incr.AddColumn("f", "q", 5l);
+  EXPECT_EQ(true, incr.HasFamilies());
+}
+
+TEST(Increment, Add) {
+  CellType cell_type = CellType::PUT;
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+
+  // add first cell
+  Increment incr{"row"};
+  incr.Add(std::move(cell));
+  EXPECT_EQ(1, incr.FamilyMap().size());
+  EXPECT_EQ(1, incr.FamilyMap().at(family).size());
+
+  // add a non-matching row
+  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+  Increment incr2{"foo"};
+  ASSERT_THROW(incr2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
+
+  // add a second cell with same family
+  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
+  incr.Add(std::move(cell3));
+  EXPECT_EQ(1, incr.FamilyMap().size());
+  EXPECT_EQ(2, incr.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
+  incr.Add(std::move(cell4));
+  EXPECT_EQ(2, incr.FamilyMap().size());
+  EXPECT_EQ(1, incr.FamilyMap().at("family-2").size());
+}
+
+TEST(Increment, AddColumn) {
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+
+  Increment incr{"row"};
+  incr.AddColumn(family, column, 5l);
+  EXPECT_EQ(1, incr.FamilyMap().size());
+  EXPECT_EQ(1, incr.FamilyMap().at(family).size());
+
+  // add a second cell with same family
+  incr.AddColumn(family, "column-2", 6l);
+  EXPECT_EQ(1, incr.FamilyMap().size());
+  EXPECT_EQ(2, incr.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  incr.AddColumn("family-2", column, 7l);
+  EXPECT_EQ(2, incr.FamilyMap().size());
+  EXPECT_EQ(1, incr.FamilyMap().at("family-2").size());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/increment.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/increment.cc b/hbase-native-client/src/hbase/client/increment.cc
new file mode 100644
index 0000000..a8ac49a
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/increment.cc
@@ -0,0 +1,56 @@
+
+
+/*
+ * 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 "hbase/client/increment.h"
+#include <folly/Conv.h>
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+#include <utility>
+
+#include "hbase/utils/bytes-util.h"
+
+namespace hbase {
+
+/**
+ *  @brief Increment the column from the specific family with the specified qualifier
+ * by the specified amount.
+ *  @param family family name
+ *  @param qualifier column qualifier
+ *  @param amount amount to increment by
+ */
+Increment& Increment::AddColumn(const std::string& family, const std::string& qualifier,
+                                int64_t amount) {
+  family_map_[family].push_back(std::move(std::make_unique<Cell>(
+      row_, family, qualifier, timestamp_, BytesUtil::ToString(amount), hbase::CellType::PUT)));
+  return *this;
+}
+Increment& Increment::Add(std::unique_ptr<Cell> cell) {
+  if (cell->Row() != row_) {
+    throw std::runtime_error("The row in " + cell->DebugString() +
+                             " doesn't match the original one " + row_);
+  }
+
+  family_map_[cell->Family()].push_back(std::move(cell));
+  return *this;
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/keyvalue-codec.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/keyvalue-codec.cc b/hbase-native-client/src/hbase/client/keyvalue-codec.cc
new file mode 100644
index 0000000..9b6e689
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/keyvalue-codec.cc
@@ -0,0 +1,69 @@
+/*
+ * 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 "hbase/client/keyvalue-codec.h"
+
+#include <string>
+
+namespace hbase {
+
+KeyValueCodec::KVDecoder::KVDecoder(std::unique_ptr<folly::IOBuf> cell_block, uint32_t offset,
+                                    uint32_t length)
+    : cell_block_(std::move(cell_block)), offset_(offset), length_(length) {}
+
+KeyValueCodec::KVDecoder::~KVDecoder() {}
+
+std::shared_ptr<Cell> KeyValueCodec::KVDecoder::Decode(folly::io::Cursor &cursor) {
+  uint32_t key_length = cursor.readBE<uint32_t>();
+  uint32_t value_length = cursor.readBE<uint32_t>();
+  uint16_t row_length = cursor.readBE<uint16_t>();
+  std::string row = cursor.readFixedString(row_length);
+  uint8_t column_family_length = cursor.readBE<uint8_t>();
+  std::string column_family = cursor.readFixedString(column_family_length);
+  int qualifier_length =
+      key_length - (row_length + column_family_length + kHBaseSizeOfKeyInfrastructure_);
+  std::string column_qualifier = cursor.readFixedString(qualifier_length);
+  uint64_t timestamp = cursor.readBE<uint64_t>();
+  uint8_t key_type = cursor.readBE<uint8_t>();
+  std::string value = cursor.readFixedString(value_length);
+
+  return std::make_shared<Cell>(row, column_family, column_qualifier, timestamp, value,
+                                static_cast<hbase::CellType>(key_type));
+}
+
+bool KeyValueCodec::KVDecoder::Advance() {
+  if (end_of_cell_block_) {
+    return false;
+  }
+
+  if (cur_pos_ == length_) {
+    end_of_cell_block_ = true;
+    return false;
+  }
+
+  folly::io::Cursor cursor(cell_block_.get());
+  cursor.skip(offset_ + cur_pos_);
+  uint32_t current_cell_size = cursor.readBE<uint32_t>();
+  current_cell_ = Decode(cursor);
+  cur_pos_ += kHBaseSizeOfInt_ + current_cell_size;
+  return true;
+}
+
+uint32_t KeyValueCodec::KVDecoder::CellBlockLength() const { return length_; }
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/load-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/load-client.cc b/hbase-native-client/src/hbase/client/load-client.cc
new file mode 100644
index 0000000..a321845
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/load-client.cc
@@ -0,0 +1,390 @@
+/*
+ * 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 <folly/Logging.h>
+#include <folly/Random.h>
+#include <gflags/gflags.h>
+
+#include <atomic>
+#include <chrono>
+#include <iostream>
+#include <thread>
+
+#include "hbase/client/client.h"
+#include "hbase/client/get.h"
+#include "hbase/client/put.h"
+#include "hbase/client/table.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Client;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::Put;
+using hbase::Table;
+using hbase::pb::TableName;
+using hbase::TimeUtil;
+using folly::Random;
+
+DEFINE_string(table, "load_test_table", "What table to do the reads and writes with");
+DEFINE_string(families, "f", "comma separated list of column family names");
+DEFINE_string(conf, "", "Conf directory to read the config from (optional)");
+DEFINE_string(zookeeper, "localhost:2181", "What zk quorum to talk to");
+DEFINE_string(znode, "/hbase", "parent znode");
+DEFINE_uint64(num_rows, 1'000'000, "How many rows to write and read");
+DEFINE_uint64(num_cols, 1000, "How many columns there are in a row");
+DEFINE_int32(threads, 10, "How many client threads");
+DEFINE_int32(batch_num_rows, 100, "number of rows in one multi-get / multi-put");
+DEFINE_uint64(report_num_rows, 5000, "How frequent we should report the progress");
+DEFINE_bool(gets, true, "perform gets");
+DEFINE_bool(scans, true, "perform scans");
+DEFINE_bool(puts, true, "perform put's");
+DEFINE_bool(appends, true, "perform append's");
+
+static constexpr const char *kNumColumn = "num";
+static constexpr const char *incrPrefix = "i";
+static constexpr const char *appendPrefix = "a";
+
+std::string PrefixZero(int total_width, int num) {
+  std::string str = std::to_string(num);
+  int prefix_len = total_width - str.length();
+  if (prefix_len > 0) {
+    return std::string(prefix_len, '0') + str;
+  }
+  return str;
+}
+
+bool Verify(std::shared_ptr<hbase::Result> result, std::string family, int m) {
+  auto col = std::to_string(m);
+  if (!result->Value(family, col)) {
+    LOG(ERROR) << "Column:" << col << " is not found for " << result->Row();
+    return false;
+  }
+  auto l = *(result->Value(family, col));
+  if (l != col) {
+    LOG(ERROR) << "value " << *(result->Value(family, "1")) << " is not " << col;
+    return false;
+  }
+  if (FLAGS_appends) {
+    if (!result->Value(family, incrPrefix + col)) {
+      LOG(ERROR) << "Column:" << (incrPrefix + col) << " is not found for " << result->Row();
+      return false;
+    }
+    auto int_val = hbase::BytesUtil::ToInt64(*(result->Value(family, incrPrefix + col)));
+    if (int_val != m) {
+      LOG(ERROR) << "value is not " << col << " for " << result->Row();
+      return false;
+    }
+    if (!result->Value(family, appendPrefix + col)) {
+      LOG(ERROR) << "Column:" << (appendPrefix + col) << " is not found for " << result->Row();
+      return false;
+    }
+    l = *(result->Value(family, appendPrefix + col));
+    if (l != col) {
+      LOG(ERROR) << "value " << *(result->Value(family, "1")) << " is not " << col;
+      return false;
+    }
+  }
+
+  return true;
+}
+
+bool Verify(std::shared_ptr<hbase::Result> result, const std::string &row,
+            const std::vector<std::string> &families) {
+  if (result == nullptr || result->IsEmpty()) {
+    LOG(ERROR) << "didn't get result";
+    return false;
+  }
+  if (result->Row().compare(row) != 0) {
+    LOG(ERROR) << "row " << result->Row() << " is not the expected: " << row;
+    return false;
+  }
+  // Test the values
+  for (auto family : families) {
+    if (!result->Value(family, kNumColumn)) {
+      LOG(ERROR) << "Column:" << kNumColumn << " is not found for " << result->Row();
+      return false;
+    }
+    auto cols = std::stoi(*(result->Value(family, kNumColumn)));
+    VLOG(3) << "Result for row:" << row << " contains " << std::to_string(cols) << " columns";
+    for (int m = 1; m <= cols; m++) {
+      if (!Verify(result, family, m)) return false;
+    }
+  }
+  return true;
+}
+
+bool DoScan(int iteration, uint64_t max_row, uint64_t rows, std::unique_ptr<Table> table,
+            const std::vector<std::string> &families) {
+  hbase::Scan scan{};
+  auto start = iteration * rows;
+  auto end = start + rows;
+  auto width = std::to_string(max_row).length();
+  scan.SetStartRow(PrefixZero(width, start));
+  if (end != max_row && end != max_row + 1) {
+    scan.SetStopRow(PrefixZero(width, end));
+  }
+
+  auto start_ns = TimeUtil::GetNowNanos();
+  auto scanner = table->Scan(scan);
+
+  auto cnt = 0;
+  auto r = scanner->Next();
+  while (r != nullptr) {
+    auto row = PrefixZero(width, start + cnt);
+    if (!Verify(r, row, families)) {
+      return false;
+    }
+    cnt++;
+    r = scanner->Next();
+    if (cnt != 0 && cnt % FLAGS_report_num_rows == 0) {
+      LOG(INFO) << "(Thread " << iteration << ") "
+                << "Scan iterated over " << cnt << " results in "
+                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+    }
+  }
+  if (cnt != rows) {
+    LOG(ERROR) << "(Thread " << iteration << ") "
+               << "Expected number of results does not match. expected:" << rows
+               << ", actual:" << cnt;
+    return false;
+  }
+  LOG(INFO) << "(Thread " << iteration << ") "
+            << "scanned " << std::to_string(cnt) << " rows in " << TimeUtil::ElapsedMillis(start_ns)
+            << " ms.";
+  return true;
+}
+
+bool DoGet(int iteration, uint64_t max_row, uint64_t rows, std::unique_ptr<Table> table,
+           const std::vector<std::string> &families, uint64_t batch_num_rows) {
+  auto width = std::to_string(max_row).length();
+  auto start_ns = TimeUtil::GetNowNanos();
+  for (uint64_t k = iteration; k <= max_row;) {
+    uint64_t total_read = 0;
+    std::vector<hbase::Get> gets;
+    for (uint64_t i = 0; i < batch_num_rows && k <= max_row; ++i, k += FLAGS_threads) {
+      std::string row = PrefixZero(width, k);
+      hbase::Get get(row);
+      gets.push_back(get);
+    }
+    VLOG(3) << "getting for " << batch_num_rows << " rows";
+    auto results = table->Get(gets);
+    if (results.size() != gets.size()) {
+      LOG(ERROR) << "(Thread " << iteration << ") "
+                 << "Expected number of results does not match. expected:" << gets.size()
+                 << ", actual:" << results.size();
+      return false;
+    }
+    for (uint64_t i = 0; i < batch_num_rows && i < results.size(); ++i) {
+      if (!Verify(results[i], gets[i].row(), families)) {
+        return false;
+      }
+    }
+    total_read += gets.size();
+    if (total_read != 0 && total_read % FLAGS_report_num_rows == 0) {
+      LOG(INFO) << "(Thread " << iteration << ") "
+                << "Sent  " << total_read << " Multi-Get requests in "
+                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+    }
+    k += batch_num_rows;
+  }
+  LOG(INFO) << "(Thread " << iteration << ") "
+            << "Sent " << rows << " gets"
+            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  return true;
+}
+
+void DoPut(int iteration, uint64_t max_row, uint64_t rows, int cols, std::unique_ptr<Table> table,
+           const std::vector<std::string> &families) {
+  auto start_ns = TimeUtil::GetNowNanos();
+  auto width = std::to_string(max_row).length();
+  for (uint64_t j = 0; j < rows; j++) {
+    std::string row = PrefixZero(width, iteration * rows + j);
+    auto put = Put{row};
+    for (auto family : families) {
+      auto n_cols = Random::rand32(1, cols);
+      put.AddColumn(family, kNumColumn, std::to_string(n_cols));
+      for (unsigned int k = 1; k <= n_cols; k++) {
+        put.AddColumn(family, std::to_string(k), std::to_string(k));
+      }
+    }
+    table->Put(put);
+    if ((j + 1) % FLAGS_report_num_rows == 0) {
+      LOG(INFO) << "(Thread " << iteration << ") "
+                << "Written " << std::to_string(j + 1) << " rows in "
+                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+    }
+  }
+  LOG(INFO) << "(Thread " << iteration << ") "
+            << "written " << std::to_string(rows) << " rows"
+            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+}
+
+bool DoAppendIncrement(int iteration, uint64_t max_row, uint64_t rows, int cols,
+                       std::unique_ptr<Table> table, const std::vector<std::string> &families) {
+  auto start_ns = TimeUtil::GetNowNanos();
+  auto width = std::to_string(max_row).length();
+  for (uint64_t j = 0; j < rows; j++) {
+    std::string row = PrefixZero(width, iteration * rows + j);
+    hbase::Get get(row);
+    auto result = table->Get(get);
+    for (auto family : families) {
+      auto n_cols = std::stoi(*(result->Value(family, kNumColumn)));
+      for (unsigned int k = 1; k <= n_cols; k++) {
+        table->Increment(
+            hbase::Increment{row}.AddColumn(family, incrPrefix + std::to_string(k), k));
+        if (!table->Append(hbase::Append{row}.Add(family, appendPrefix + std::to_string(k),
+                                                  std::to_string(k)))) {
+          LOG(ERROR) << "(Thread " << iteration << ") "
+                     << "append for " << row << " family: " << family << " failed";
+          return false;
+        }
+      }
+    }
+    if ((j + 1) % FLAGS_report_num_rows == 0)
+      LOG(INFO) << "(Thread " << iteration << ") "
+                << "Written " << std::to_string(j + 1) << " increments"
+                << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+  LOG(INFO) << "(Thread " << iteration << ") "
+            << "written " << std::to_string(rows) << " increments"
+            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  return true;
+}
+
+int main(int argc, char *argv[]) {
+  gflags::SetUsageMessage("Load client to manipulate multiple rows from HBase on the comamnd line");
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  google::InitGoogleLogging(argv[0]);
+  google::InstallFailureSignalHandler();
+  FLAGS_logtostderr = 1;
+  FLAGS_stderrthreshold = 1;
+
+  if (FLAGS_batch_num_rows < 1) {
+    LOG(ERROR) << "size of multi get should be positive";
+    return -1;
+  }
+  if (!FLAGS_gets && !FLAGS_scans && !FLAGS_puts) {
+    LOG(ERROR) << "Must perform at least Get or Put operations";
+    return -1;
+  }
+  std::shared_ptr<Configuration> conf = nullptr;
+  if (FLAGS_conf == "") {
+    // Configuration
+    conf = std::make_shared<Configuration>();
+    conf->Set("hbase.zookeeper.quorum", FLAGS_zookeeper);
+    conf->Set("zookeeper.znode.parent", FLAGS_znode);
+  } else {
+    setenv("HBASE_CONF", FLAGS_conf.c_str(), 1);
+    hbase::HBaseConfigurationLoader loader;
+    conf = std::make_shared<Configuration>(loader.LoadDefaultResources().value());
+  }
+  auto tn = std::make_shared<TableName>(folly::to<TableName>(FLAGS_table));
+  auto num_puts = FLAGS_num_rows;
+
+  auto client = std::make_unique<Client>(*conf);
+
+  // Do the Put requests
+
+  std::vector<std::string> families;
+  std::size_t pos = 0, found;
+  while ((found = FLAGS_families.find_first_of(',', pos)) != std::string::npos) {
+    families.push_back(FLAGS_families.substr(pos, found - pos));
+    pos = found + 1;
+  }
+  families.push_back(FLAGS_families.substr(pos));
+
+  int rows = FLAGS_num_rows / FLAGS_threads;
+  if (FLAGS_num_rows % FLAGS_threads != 0) rows++;
+  int cols = FLAGS_num_cols;
+  std::atomic<int8_t> succeeded{1};  // not using bool since we want atomic &=
+  if (FLAGS_puts) {
+    LOG(INFO) << "Sending put requests";
+    auto start_ns = TimeUtil::GetNowNanos();
+    std::vector<std::thread> writer_threads;
+    for (int i = 0; i < FLAGS_threads; i++) {
+      writer_threads.push_back(std::thread([&, i] {
+        auto table = client->Table(*tn);
+        DoPut(i, FLAGS_num_rows - 1, rows, cols, std::move(table), families);
+      }));
+    }
+    for (auto &t : writer_threads) {
+      t.join();
+    }
+    LOG(INFO) << "Successfully sent  " << num_puts << " Put requests in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+  if (FLAGS_appends) {
+    LOG(INFO) << "Sending append/increment requests";
+    auto start_ns = TimeUtil::GetNowNanos();
+    std::vector<std::thread> writer_threads;
+    for (int i = 0; i < FLAGS_threads; i++) {
+      writer_threads.push_back(std::thread([&, i] {
+        auto table = client->Table(*tn);
+        succeeded &=
+            DoAppendIncrement(i, FLAGS_num_rows - 1, rows, cols, std::move(table), families);
+      }));
+    }
+    for (auto &t : writer_threads) {
+      t.join();
+    }
+    LOG(INFO) << "Successfully sent  " << num_puts << " append requests in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+
+  if (FLAGS_scans) {
+    LOG(INFO) << "Sending scan requests";
+    auto start_ns = TimeUtil::GetNowNanos();
+    std::vector<std::thread> reader_threads;
+    for (int i = 0; i < FLAGS_threads; i++) {
+      reader_threads.push_back(std::thread([&, i] {
+        auto table1 = client->Table(*tn);
+        succeeded &= DoScan(i, FLAGS_num_rows - 1, rows, std::move(table1), families);
+      }));
+    }
+    for (auto &t : reader_threads) {
+      t.join();
+    }
+
+    LOG(INFO) << (succeeded.load() ? "Successfully " : "Failed. ") << " scannned " << num_puts
+              << " rows in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+
+  if (FLAGS_gets) {
+    LOG(INFO) << "Sending get requests";
+    auto start_ns = TimeUtil::GetNowNanos();
+    std::vector<std::thread> reader_threads;
+    for (int i = 0; i < FLAGS_threads; i++) {
+      reader_threads.push_back(std::thread([&, i] {
+        auto table1 = client->Table(*tn);
+        succeeded &=
+            DoGet(i, FLAGS_num_rows - 1, rows, std::move(table1), families, FLAGS_batch_num_rows);
+      }));
+    }
+    for (auto &t : reader_threads) {
+      t.join();
+    }
+
+    LOG(INFO) << (succeeded.load() ? "Successful. " : "Failed. ") << " sent multi-get requests for "
+              << num_puts << " rows in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+  client->Close();
+
+  return succeeded.load() ? 0 : -1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/location-cache-retry-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/location-cache-retry-test.cc b/hbase-native-client/src/hbase/client/location-cache-retry-test.cc
new file mode 100644
index 0000000..283cf85
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/location-cache-retry-test.cc
@@ -0,0 +1,112 @@
+/*
+ * 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 <gtest/gtest.h>
+
+#include "hbase/client/append.h"
+#include "hbase/client/cell.h"
+#include "hbase/client/client.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/delete.h"
+#include "hbase/client/get.h"
+#include "hbase/client/hbase-configuration-loader.h"
+#include "hbase/client/increment.h"
+#include "hbase/client/meta-utils.h"
+#include "hbase/client/put.h"
+#include "hbase/client/result.h"
+#include "hbase/client/table.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/test-util/test-util.h"
+#include "hbase/utils/bytes-util.h"
+
+using hbase::Cell;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::MetaUtil;
+using hbase::RetriesExhaustedException;
+using hbase::Put;
+using hbase::Table;
+using hbase::TestUtil;
+
+using std::chrono_literals::operator"" s;
+
+class LocationCacheRetryTest : public ::testing::Test {
+ public:
+  static std::unique_ptr<hbase::TestUtil> test_util;
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util = std::make_unique<hbase::TestUtil>();
+    test_util->StartMiniCluster(2);
+    test_util->conf()->SetInt("hbase.client.retries.number", 5);
+  }
+};
+
+std::unique_ptr<hbase::TestUtil> LocationCacheRetryTest::test_util = nullptr;
+
+TEST_F(LocationCacheRetryTest, GetFromMetaTable) {
+  auto tn = folly::to<hbase::pb::TableName>("hbase:meta");
+  auto row = "test1";
+
+  hbase::Client client(*LocationCacheRetryTest::test_util->conf());
+
+  // do a get against the other table, but not the actual table "t".
+  auto table = client.Table(tn);
+  hbase::Get get(row);
+  auto result = table->Get(get);
+
+  LocationCacheRetryTest::test_util->MoveRegion(MetaUtil::kMetaRegion, "");
+
+  std::this_thread::sleep_for(3s);  // sleep 3 sec
+
+  result = table->Get(get);
+}
+
+TEST_F(LocationCacheRetryTest, PutGet) {
+  LocationCacheRetryTest::test_util->CreateTable("t", "d");
+  LocationCacheRetryTest::test_util->CreateTable("t2", "d");
+
+  auto tn = folly::to<hbase::pb::TableName>("t");
+  auto tn2 = folly::to<hbase::pb::TableName>("t2");
+  auto row = "test1";
+
+  hbase::Client client(*LocationCacheRetryTest::test_util->conf());
+
+  // do a get against the other table, but not the actual table "t".
+  auto table = client.Table(tn);
+  auto table2 = client.Table(tn2);
+  hbase::Get get(row);
+  auto result = table2->Get(get);
+
+  // we should have already cached the location of meta right now. Now
+  // move the meta region to the other server so that we will get a NotServingRegionException
+  // when we do the actual location lookup request. If there is no invalidation
+  // of the meta's own location, then following put/get will result in retries exhausted.
+  LocationCacheRetryTest::test_util->MoveRegion(MetaUtil::kMetaRegion, "");
+
+  std::this_thread::sleep_for(3s);  // sleep 3 sec
+
+  table->Put(Put{row}.AddColumn("d", "1", "value1"));
+
+  result = table->Get(get);
+
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ("test1", result->Row());
+  EXPECT_EQ("value1", *(result->Value("d", "1")));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/location-cache-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/location-cache-test.cc b/hbase-native-client/src/hbase/client/location-cache-test.cc
new file mode 100644
index 0000000..af25902
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/location-cache-test.cc
@@ -0,0 +1,164 @@
+/*
+ * 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 "hbase/client/location-cache.h"
+
+#include <folly/Memory.h>
+#include <gtest/gtest.h>
+
+#include <chrono>
+
+#include "hbase/client/keyvalue-codec.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/test-util/test-util.h"
+
+using hbase::Cell;
+using hbase::Configuration;
+using hbase::ConnectionPool;
+using hbase::MetaUtil;
+using hbase::LocationCache;
+using hbase::TestUtil;
+using hbase::KeyValueCodec;
+using std::chrono::milliseconds;
+
+class LocationCacheTest : public ::testing::Test {
+ protected:
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util_ = std::make_unique<TestUtil>();
+    test_util_->StartMiniCluster(2);
+  }
+  static void TearDownTestCase() { test_util_.release(); }
+
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+
+ public:
+  static std::unique_ptr<TestUtil> test_util_;
+};
+
+std::unique_ptr<TestUtil> LocationCacheTest::test_util_ = nullptr;
+
+TEST_F(LocationCacheTest, TestGetMetaNodeContents) {
+  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
+  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
+  auto codec = std::make_shared<KeyValueCodec>();
+  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
+  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
+  auto f = cache.LocateMeta();
+  auto result = f.get();
+  ASSERT_FALSE(f.hasException());
+  ASSERT_TRUE(result.has_port());
+  ASSERT_TRUE(result.has_host_name());
+  cpu->stop();
+  io->stop();
+}
+
+TEST_F(LocationCacheTest, TestGetRegionLocation) {
+  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
+  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
+  auto codec = std::make_shared<KeyValueCodec>();
+  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
+  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
+
+  // If there is no table this should throw an exception
+  auto tn = folly::to<hbase::pb::TableName>("t");
+  auto row = "test";
+  ASSERT_ANY_THROW(cache.LocateFromMeta(tn, row).get(milliseconds(1000)));
+  LocationCacheTest::test_util_->CreateTable("t", "d");
+  auto loc = cache.LocateFromMeta(tn, row).get(milliseconds(1000));
+  ASSERT_TRUE(loc != nullptr);
+  cpu->stop();
+  io->stop();
+}
+
+TEST_F(LocationCacheTest, TestCaching) {
+  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
+  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
+  auto codec = std::make_shared<KeyValueCodec>();
+  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
+  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
+
+  auto tn_1 = folly::to<hbase::pb::TableName>("t1");
+  auto tn_2 = folly::to<hbase::pb::TableName>("t2");
+  auto tn_3 = folly::to<hbase::pb::TableName>("t3");
+  auto row_a = "a";
+
+  // test location pulled from meta gets cached
+  ASSERT_ANY_THROW(cache.LocateRegion(tn_1, row_a).get(milliseconds(1000)));
+  ASSERT_ANY_THROW(cache.LocateFromMeta(tn_1, row_a).get(milliseconds(1000)));
+  LocationCacheTest::test_util_->CreateTable("t1", "d");
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_1, row_a));
+  auto loc = cache.LocateRegion(tn_1, row_a).get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_1, row_a));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_1, row_a));
+
+  // test with two regions
+  std::vector<std::string> keys;
+  keys.push_back("b");
+  LocationCacheTest::test_util_->CreateTable("t2", "d", keys);
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_2, "a"));
+  loc = cache.LocateRegion(tn_2, "a").get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_2, "a"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "a"));
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_2, "b"));
+  loc = cache.LocateRegion(tn_2, "b").get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_2, "b"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "b"));
+  ASSERT_TRUE(cache.IsLocationCached(tn_2, "ba"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "ba"));
+
+  // test with three regions
+  keys.clear();
+  keys.push_back("b");
+  keys.push_back("c");
+  LocationCacheTest::test_util_->CreateTable("t3", "d", keys);
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_3, "c"));
+  ASSERT_FALSE(cache.IsLocationCached(tn_3, "ca"));
+  loc = cache.LocateRegion(tn_3, "ca").get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "c"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "c"));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "ca"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "ca"));
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_3, "b"));
+  loc = cache.LocateRegion(tn_3, "b").get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "b"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "b"));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "ba"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "ba"));
+
+  // clear second region
+  cache.ClearCachedLocation(tn_3, "b");
+  ASSERT_FALSE(cache.IsLocationCached(tn_3, "b"));
+
+  ASSERT_FALSE(cache.IsLocationCached(tn_3, "a"));
+  loc = cache.LocateRegion(tn_3, "a").get(milliseconds(1000));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "a"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "a"));
+  ASSERT_TRUE(cache.IsLocationCached(tn_3, "abc"));
+  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "abc"));
+
+  cpu->stop();
+  io->stop();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/location-cache.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/location-cache.cc b/hbase-native-client/src/hbase/client/location-cache.cc
new file mode 100644
index 0000000..2be2b94
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/location-cache.cc
@@ -0,0 +1,334 @@
+/*
+ * 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 "hbase/client/location-cache.h"
+
+#include <folly/Conv.h>
+#include <folly/Logging.h>
+#include <folly/io/IOBuf.h>
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+
+#include <map>
+#include <shared_mutex>
+#include <utility>
+
+#include "hbase/connection/response.h"
+#include "hbase/connection/rpc-connection.h"
+#include "hbase/client/meta-utils.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/ZooKeeper.pb.h"
+#include "hbase/serde/region-info.h"
+#include "hbase/serde/server-name.h"
+#include "hbase/serde/zk.h"
+
+using hbase::pb::MetaRegionServer;
+using hbase::pb::ServerName;
+using hbase::pb::TableName;
+
+namespace hbase {
+
+LocationCache::LocationCache(std::shared_ptr<hbase::Configuration> conf,
+                             std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                             std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                             std::shared_ptr<ConnectionPool> cp)
+    : conf_(conf),
+      io_executor_(io_executor),
+      cpu_executor_(cpu_executor),
+      cp_(cp),
+      meta_promise_(nullptr),
+      meta_lock_(),
+      meta_util_(),
+      zk_(nullptr),
+      cached_locations_(),
+      locations_lock_() {
+  zk_quorum_ = ZKUtil::ParseZooKeeperQuorum(*conf_);
+  EnsureZooKeeperConnection();
+}
+
+LocationCache::~LocationCache() { CloseZooKeeperConnection(); }
+
+void LocationCache::CloseZooKeeperConnection() {
+  if (zk_ != nullptr) {
+    zookeeper_close(zk_);
+    zk_ = nullptr;
+    LOG(INFO) << "Closed connection to ZooKeeper.";
+  }
+}
+
+void LocationCache::EnsureZooKeeperConnection() {
+  if (zk_ == nullptr) {
+    LOG(INFO) << "Connecting to ZooKeeper. Quorum:" + zk_quorum_;
+    auto session_timeout = ZKUtil::SessionTimeout(*conf_);
+    zk_ = zookeeper_init(zk_quorum_.c_str(), nullptr, session_timeout, nullptr, nullptr, 0);
+  }
+}
+
+folly::Future<ServerName> LocationCache::LocateMeta() {
+  std::lock_guard<std::recursive_mutex> g(meta_lock_);
+  if (meta_promise_ == nullptr) {
+    this->RefreshMetaLocation();
+  }
+  return meta_promise_->getFuture().onError([&](const folly::exception_wrapper &ew) {
+    auto promise = InvalidateMeta();
+    if (promise) {
+      promise->setException(ew);
+    }
+    throw ew;
+    return ServerName{};
+  });
+}
+
+std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> LocationCache::InvalidateMeta() {
+  VLOG(2) << "Invalidating meta location";
+  std::lock_guard<std::recursive_mutex> g(meta_lock_);
+  if (meta_promise_ != nullptr) {
+    // return the unique_ptr back to the caller.
+    std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> ret = nullptr;
+    std::swap(ret, meta_promise_);
+    return ret;
+  } else {
+    return nullptr;
+  }
+}
+
+void LocationCache::RefreshMetaLocation() {
+  meta_promise_ = std::make_shared<folly::SharedPromise<ServerName>>();
+  auto p = meta_promise_;
+  cpu_executor_->add([this, p] {
+    std::lock_guard<std::recursive_mutex> g(meta_lock_);
+    p->setWith([&] { return this->ReadMetaLocation(); });
+  });
+}
+
+// Note: this is a blocking call to zookeeper
+ServerName LocationCache::ReadMetaLocation() {
+  auto buf = folly::IOBuf::create(4096);
+  ZkDeserializer derser;
+  EnsureZooKeeperConnection();
+
+  // This needs to be int rather than size_t as that's what ZK expects.
+  int len = buf->capacity();
+  std::string zk_node = ZKUtil::MetaZNode(*conf_);
+  int zk_result = zoo_get(this->zk_, zk_node.c_str(), 0,
+                          reinterpret_cast<char *>(buf->writableData()), &len, nullptr);
+  if (zk_result != ZOK || len < 9) {
+    LOG(ERROR) << "Error getting meta location.";
+    // We just close the zk connection, and let the upper levels retry.
+    CloseZooKeeperConnection();
+    throw std::runtime_error("Error getting meta location. Quorum: " + zk_quorum_);
+  }
+  buf->append(len);
+
+  MetaRegionServer mrs;
+  if (derser.Parse(buf.get(), &mrs) == false) {
+    LOG(ERROR) << "Unable to decode";
+    throw std::runtime_error("Error getting meta location (Unable to decode). Quorum: " +
+                             zk_quorum_);
+  }
+  return mrs.server();
+}
+
+folly::Future<std::shared_ptr<RegionLocation>> LocationCache::LocateFromMeta(
+    const TableName &tn, const std::string &row) {
+  return this->LocateMeta()
+      .via(cpu_executor_.get())
+      .then([this](ServerName sn) {
+        // TODO: use RpcClient?
+        auto remote_id = std::make_shared<ConnectionId>(sn.host_name(), sn.port());
+        return this->cp_->GetConnection(remote_id);
+      })
+      .then([tn, row, this](std::shared_ptr<RpcConnection> rpc_connection) {
+        return rpc_connection->SendRequest(std::move(meta_util_.MetaRequest(tn, row)));
+      })
+      .onError([&](const folly::exception_wrapper &ew) {
+        auto promise = InvalidateMeta();
+        throw ew;
+        return static_cast<std::unique_ptr<Response>>(nullptr);
+      })
+      .then([tn, this](std::unique_ptr<Response> resp) {
+        // take the protobuf response and make it into
+        // a region location.
+        return meta_util_.CreateLocation(std::move(*resp), tn);
+      })
+      .then([tn, this](std::shared_ptr<RegionLocation> rl) {
+        // Make sure that the correct location was found.
+        if (rl->region_info().table_name().namespace_() != tn.namespace_() ||
+            rl->region_info().table_name().qualifier() != tn.qualifier()) {
+          throw TableNotFoundException(folly::to<std::string>(tn));
+        }
+        return rl;
+      })
+      .then([this](std::shared_ptr<RegionLocation> rl) {
+        auto remote_id =
+            std::make_shared<ConnectionId>(rl->server_name().host_name(), rl->server_name().port());
+        return rl;
+      })
+      .then([tn, this](std::shared_ptr<RegionLocation> rl) {
+        // now add fetched location to the cache.
+        this->CacheLocation(tn, rl);
+        return rl;
+      });
+}
+
+constexpr const char *MetaUtil::kMetaRegionName;
+
+folly::Future<std::shared_ptr<RegionLocation>> LocationCache::LocateRegion(
+    const TableName &tn, const std::string &row, const RegionLocateType locate_type,
+    const int64_t locate_ns) {
+  // We maybe asked to locate meta itself
+  if (MetaUtil::IsMeta(tn)) {
+    return LocateMeta().then([this](const ServerName &server_name) {
+      auto rl = std::make_shared<RegionLocation>(MetaUtil::kMetaRegionName,
+                                                 meta_util_.meta_region_info(), server_name);
+      return rl;
+    });
+  }
+
+  // TODO: implement region locate type and timeout
+  auto cached_loc = this->GetCachedLocation(tn, row);
+  if (cached_loc != nullptr) {
+    return cached_loc;
+  } else {
+    return this->LocateFromMeta(tn, row);
+  }
+}
+
+// must hold shared lock on locations_lock_
+std::shared_ptr<RegionLocation> LocationCache::GetCachedLocation(const hbase::pb::TableName &tn,
+                                                                 const std::string &row) {
+  auto t_locs = this->GetTableLocations(tn);
+  std::shared_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
+
+  // looking for the "floor" key as a start key
+  auto possible_region = t_locs->upper_bound(row);
+
+  if (t_locs->empty()) {
+    VLOG(5) << "Could not find region in cache, table map is empty";
+    return nullptr;
+  }
+
+  if (possible_region == t_locs->begin()) {
+    VLOG(5) << "Could not find region in cache, all keys are greater, row:" << row
+            << " ,possible_region:" << possible_region->second->DebugString();
+    return nullptr;
+  }
+  --possible_region;
+
+  VLOG(5) << "Found possible region in cache for row:" << row
+          << " ,possible_region:" << possible_region->second->DebugString();
+
+  // found possible start key, now need to check end key
+  if (possible_region->second->region_info().end_key() == "" ||
+      possible_region->second->region_info().end_key() > row) {
+    VLOG(2) << "Found region in cache for row:" << row
+            << " ,region:" << possible_region->second->DebugString();
+    return possible_region->second;
+  } else {
+    return nullptr;
+  }
+}
+
+// must hold unique lock on locations_lock_
+void LocationCache::CacheLocation(const hbase::pb::TableName &tn,
+                                  const std::shared_ptr<RegionLocation> loc) {
+  auto t_locs = this->GetTableLocations(tn);
+  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
+
+  (*t_locs)[loc->region_info().start_key()] = loc;
+  VLOG(1) << "Cached location for region:" << loc->DebugString();
+}
+
+// must hold shared lock on locations_lock_
+bool LocationCache::IsLocationCached(const hbase::pb::TableName &tn, const std::string &row) {
+  return (this->GetCachedLocation(tn, row) != nullptr);
+}
+
+// shared lock needed for cases when this table has been requested before;
+// in the rare case it hasn't, unique lock will be grabbed to add it to cache
+std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetTableLocations(
+    const hbase::pb::TableName &tn) {
+  auto found_locs = this->GetCachedTableLocations(tn);
+  if (found_locs == nullptr) {
+    found_locs = this->GetNewTableLocations(tn);
+  }
+  return found_locs;
+}
+
+std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetCachedTableLocations(
+    const hbase::pb::TableName &tn) {
+  folly::SharedMutexWritePriority::ReadHolder r_holder{locations_lock_};
+
+  auto table_locs = cached_locations_.find(tn);
+  if (table_locs != cached_locations_.end()) {
+    return table_locs->second;
+  } else {
+    return nullptr;
+  }
+}
+
+std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetNewTableLocations(
+    const hbase::pb::TableName &tn) {
+  // double-check locking under upgradable lock
+  folly::SharedMutexWritePriority::UpgradeHolder u_holder{locations_lock_};
+
+  auto table_locs = cached_locations_.find(tn);
+  if (table_locs != cached_locations_.end()) {
+    return table_locs->second;
+  }
+  folly::SharedMutexWritePriority::WriteHolder w_holder{std::move(u_holder)};
+
+  auto t_locs_p = std::make_shared<std::map<std::string, std::shared_ptr<RegionLocation>>>();
+  cached_locations_.insert(std::make_pair(tn, t_locs_p));
+  return t_locs_p;
+}
+
+// must hold unique lock on locations_lock_
+void LocationCache::ClearCache() {
+  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
+  cached_locations_.clear();
+}
+
+// must hold unique lock on locations_lock_
+void LocationCache::ClearCachedLocations(const hbase::pb::TableName &tn) {
+  VLOG(1) << "ClearCachedLocations, table:" << folly::to<std::string>(tn);
+  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
+  cached_locations_.erase(tn);
+  if (MetaUtil::IsMeta(tn)) {
+    InvalidateMeta();
+  }
+}
+
+// must hold unique lock on locations_lock_
+void LocationCache::ClearCachedLocation(const hbase::pb::TableName &tn, const std::string &row) {
+  VLOG(1) << "ClearCachedLocation, table:" << folly::to<std::string>(tn) << ", row:" << row;
+  auto table_locs = this->GetTableLocations(tn);
+  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
+  table_locs->erase(row);
+  if (MetaUtil::IsMeta(tn)) {
+    InvalidateMeta();
+  }
+}
+
+void LocationCache::UpdateCachedLocation(const RegionLocation &loc,
+                                         const folly::exception_wrapper &error) {
+  // TODO: just clears the location for now. We can inspect RegionMovedExceptions, etc later.
+  ClearCachedLocation(loc.region_info().table_name(), loc.region_info().start_key());
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/meta-utils.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/meta-utils.cc b/hbase-native-client/src/hbase/client/meta-utils.cc
new file mode 100644
index 0000000..338c43e
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/meta-utils.cc
@@ -0,0 +1,125 @@
+/*
+ * 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 "hbase/client/meta-utils.h"
+
+#include <folly/Conv.h>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/serde/region-info.h"
+#include "hbase/serde/server-name.h"
+#include "hbase/serde/table-name.h"
+
+using hbase::pb::TableName;
+using hbase::pb::RegionInfo;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+using hbase::pb::ScanRequest;
+using hbase::pb::ServerName;
+
+namespace hbase {
+
+MetaUtil::MetaUtil() {
+  meta_region_info_.set_start_key("");
+  meta_region_info_.set_end_key("");
+  meta_region_info_.set_offline(false);
+  meta_region_info_.set_split(false);
+  meta_region_info_.set_replica_id(0);
+  meta_region_info_.set_split(false);
+  meta_region_info_.set_region_id(1);
+  meta_region_info_.mutable_table_name()->set_namespace_(MetaUtil::kSystemNamespace);
+  meta_region_info_.mutable_table_name()->set_qualifier(MetaUtil::kMetaTableQualifier);
+}
+
+std::string MetaUtil::RegionLookupRowkey(const TableName &tn, const std::string &row) const {
+  return folly::to<std::string>(tn, ",", row, ",", "999999999999999999");
+}
+
+std::unique_ptr<Request> MetaUtil::MetaRequest(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(MetaUtil::kMetaRegion);
+  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(MetaUtil::kCatalogFamily);
+  info_col->add_qualifier(MetaUtil::kServerColumn);
+  info_col->add_qualifier(MetaUtil::kRegionInfoColumn);
+
+  scan->set_start_row(RegionLookupRowkey(tn, row));
+  return request;
+}
+
+std::shared_ptr<RegionLocation> MetaUtil::CreateLocation(const Response &resp,
+                                                         const TableName &tn) {
+  std::vector<std::shared_ptr<Result>> results = ResponseConverter::FromScanResponse(resp);
+  if (results.size() == 0) {
+    throw TableNotFoundException(folly::to<std::string>(tn));
+  }
+  if (results.size() != 1) {
+    throw std::runtime_error("Was expecting exactly 1 result in meta scan response, got:" +
+                             std::to_string(results.size()));
+  }
+  auto result = *results[0];
+
+  auto region_info_str = result.Value(MetaUtil::kCatalogFamily, MetaUtil::kRegionInfoColumn);
+  auto server_str = result.Value(MetaUtil::kCatalogFamily, MetaUtil::kServerColumn);
+  CHECK(region_info_str);
+  CHECK(server_str);
+
+  auto row = result.Row();
+  auto region_info = folly::to<RegionInfo>(*region_info_str);
+  auto server_name = folly::to<ServerName>(*server_str);
+  return std::make_shared<RegionLocation>(row, std::move(region_info), server_name);
+}
+
+bool MetaUtil::IsMeta(const hbase::pb::TableName &tn) {
+  return folly::to<std::string>(tn) == MetaUtil::kMetaTableName;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/multi-response.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/multi-response.cc b/hbase-native-client/src/hbase/client/multi-response.cc
new file mode 100644
index 0000000..564e356
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/multi-response.cc
@@ -0,0 +1,86 @@
+/*
+ * 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 "hbase/client/multi-response.h"
+#include <glog/logging.h>
+#include "hbase/client/region-result.h"
+
+using hbase::pb::RegionLoadStats;
+
+namespace hbase {
+
+MultiResponse::MultiResponse() {}
+
+int MultiResponse::Size() const {
+  int size = 0;
+  for (const auto& result : results_) {
+    size += result.second->ResultOrExceptionSize();
+  }
+  return size;
+}
+
+void MultiResponse::AddRegionResult(const std::string& region_name, int32_t original_index,
+                                    std::shared_ptr<Result> result,
+                                    std::shared_ptr<folly::exception_wrapper> exc) {
+  auto itr = results_.find(region_name);
+  if (itr == results_.end()) {
+    auto region_result = std::make_shared<RegionResult>();
+    region_result->AddResultOrException(original_index, result, exc);
+    results_[region_name] = region_result;
+  } else {
+    itr->second->AddResultOrException(original_index, result, exc);
+  }
+}
+
+void MultiResponse::AddRegionException(const std::string& region_name,
+                                       std::shared_ptr<folly::exception_wrapper> exception) {
+  VLOG(8) << "Store Region Exception:- " << exception->what() << "; Region[" << region_name << "];";
+  bool region_found = false;
+  auto itr = exceptions_.find(region_name);
+  if (itr == exceptions_.end()) {
+    auto region_result = std::make_shared<folly::exception_wrapper>();
+    exceptions_[region_name] = exception;
+  } else {
+    itr->second = exception;
+  }
+}
+
+std::shared_ptr<folly::exception_wrapper> MultiResponse::RegionException(
+    const std::string& region_name) const {
+  auto find = exceptions_.at(region_name);
+  return find;
+}
+
+const std::map<std::string, std::shared_ptr<folly::exception_wrapper> >&
+MultiResponse::RegionExceptions() const {
+  return exceptions_;
+}
+
+void MultiResponse::AddStatistic(const std::string& region_name,
+                                 std::shared_ptr<RegionLoadStats> stat) {
+  results_[region_name]->set_stat(stat);
+}
+
+const std::map<std::string, std::shared_ptr<RegionResult> >& MultiResponse::RegionResults() const {
+  return results_;
+}
+
+MultiResponse::~MultiResponse() {}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/mutation.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/mutation.cc b/hbase-native-client/src/hbase/client/mutation.cc
new file mode 100644
index 0000000..5a43b2f
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/mutation.cc
@@ -0,0 +1,69 @@
+
+
+/*
+ * 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 "hbase/client/mutation.h"
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+
+Mutation::Mutation(const std::string &row) : Row(row) {}
+Mutation::Mutation(const std::string &row, int64_t timestamp) : Row(row), timestamp_(timestamp) {}
+
+Mutation::Mutation(const Mutation &mutation) {
+  row_ = mutation.row_;
+  durability_ = mutation.durability_;
+  timestamp_ = mutation.timestamp_;
+  for (auto const &e : mutation.family_map_) {
+    for (auto const &c : e.second) {
+      family_map_[e.first].push_back(std::make_unique<Cell>(*c));
+    }
+  }
+}
+
+Mutation &Mutation::operator=(const Mutation &mutation) {
+  row_ = mutation.row_;
+  durability_ = mutation.durability_;
+  timestamp_ = mutation.timestamp_;
+  for (auto const &e : mutation.family_map_) {
+    for (auto const &c : e.second) {
+      family_map_[e.first].push_back(std::make_unique<Cell>(*c));
+    }
+  }
+  return *this;
+}
+
+pb::MutationProto_Durability Mutation::Durability() const { return durability_; }
+
+Mutation &Mutation::SetDurability(pb::MutationProto_Durability durability) {
+  durability_ = durability;
+  return *this;
+}
+
+bool Mutation::HasFamilies() const { return !family_map_.empty(); }
+
+std::unique_ptr<Cell> Mutation::CreateCell(const std::string &family, const std::string &qualifier,
+                                           int64_t timestamp, const std::string &value) {
+  return std::make_unique<Cell>(row_, family, qualifier, timestamp, value, hbase::CellType::PUT);
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/put-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/put-test.cc b/hbase-native-client/src/hbase/client/put-test.cc
new file mode 100644
index 0000000..0657bb6
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/put-test.cc
@@ -0,0 +1,135 @@
+/*
+ * 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 <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "hbase/client/mutation.h"
+#include "hbase/client/put.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Put;
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Mutation;
+using hbase::TimeUtil;
+
+const constexpr int64_t Mutation::kLatestTimestamp;
+
+TEST(Put, Row) {
+  Put put{"foo"};
+  EXPECT_EQ("foo", put.row());
+}
+
+TEST(Put, Durability) {
+  Put put{"row"};
+  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, put.Durability());
+
+  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
+  put.SetDurability(skipWal);
+  EXPECT_EQ(skipWal, put.Durability());
+}
+
+TEST(Put, Timestamp) {
+  Put put{"row"};
+
+  // test default timestamp
+  EXPECT_EQ(Mutation::kLatestTimestamp, put.TimeStamp());
+
+  // set custom timestamp
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  put.SetTimeStamp(ts);
+  EXPECT_EQ(ts, put.TimeStamp());
+
+  // Add a column with custom timestamp
+  put.AddColumn("f", "q", "v");
+  auto &cell = put.FamilyMap().at("f")[0];
+  EXPECT_EQ(ts, cell->Timestamp());
+}
+
+TEST(Put, HasFamilies) {
+  Put put{"row"};
+
+  EXPECT_EQ(false, put.HasFamilies());
+
+  put.AddColumn("f", "q", "v");
+  EXPECT_EQ(true, put.HasFamilies());
+}
+
+TEST(Put, Add) {
+  CellType cell_type = CellType::PUT;
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+
+  // add first cell
+  Put put{"row"};
+  put.Add(std::move(cell));
+  EXPECT_EQ(1, put.FamilyMap().size());
+  EXPECT_EQ(1, put.FamilyMap().at(family).size());
+
+  // add a non-matching row
+  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+  Put put2{"foo"};
+  ASSERT_THROW(put2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
+
+  // add a second cell with same family
+  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
+  put.Add(std::move(cell3));
+  EXPECT_EQ(1, put.FamilyMap().size());
+  EXPECT_EQ(2, put.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
+  put.Add(std::move(cell4));
+  EXPECT_EQ(2, put.FamilyMap().size());
+  EXPECT_EQ(1, put.FamilyMap().at("family-2").size());
+}
+
+TEST(Put, AddColumn) {
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+
+  Put put{"row"};
+  put.AddColumn(family, column, value);
+  EXPECT_EQ(1, put.FamilyMap().size());
+  EXPECT_EQ(1, put.FamilyMap().at(family).size());
+
+  // add a second cell with same family
+  put.AddColumn(family, "column-2", value);
+  EXPECT_EQ(1, put.FamilyMap().size());
+  EXPECT_EQ(2, put.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  put.AddColumn("family-2", column, value);
+  EXPECT_EQ(2, put.FamilyMap().size());
+  EXPECT_EQ(1, put.FamilyMap().at("family-2").size());
+
+  // use the AddColumn overload
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  put.AddColumn(family, column, ts, value);
+  EXPECT_EQ(2, put.FamilyMap().size());
+  EXPECT_EQ(3, put.FamilyMap().at(family).size());
+  auto &cell = put.FamilyMap().at(family)[2];
+  EXPECT_EQ(ts, cell->Timestamp());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/put.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/put.cc b/hbase-native-client/src/hbase/client/put.cc
new file mode 100644
index 0000000..d942d3d
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/put.cc
@@ -0,0 +1,68 @@
+
+
+/*
+ * 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 "hbase/client/put.h"
+#include <folly/Conv.h>
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+#include <utility>
+
+namespace hbase {
+
+/**
+ *  @brief Add the specified column and value to this Put operation.
+ *  @param family family name
+ *  @param qualifier column qualifier
+ *  @param value column value
+ */
+Put& Put::AddColumn(const std::string& family, const std::string& qualifier,
+                    const std::string& value) {
+  return AddColumn(family, qualifier, timestamp_, value);
+}
+
+/**
+ *  @brief Add the specified column and value to this Put operation.
+ *  @param family family name
+ *  @param qualifier column qualifier
+ *  @param timestamp version timestamp
+ *  @param value column value
+ */
+Put& Put::AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp,
+                    const std::string& value) {
+  if (timestamp < 0) {
+    throw std::runtime_error("Timestamp cannot be negative. ts=" +
+                             folly::to<std::string>(timestamp));
+  }
+
+  return Add(CreateCell(family, qualifier, timestamp, value));
+}
+
+Put& Put::Add(std::unique_ptr<Cell> cell) {
+  if (cell->Row() != row_) {
+    throw std::runtime_error("The row in " + cell->DebugString() +
+                             " doesn't match the original one " + row_);
+  }
+
+  family_map_[cell->Family()].push_back(std::move(cell));
+  return *this;
+}
+}  // namespace hbase


[06/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/raw-async-table.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/raw-async-table.cc b/hbase-native-client/src/hbase/client/raw-async-table.cc
new file mode 100644
index 0000000..96361e4
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/raw-async-table.cc
@@ -0,0 +1,260 @@
+/*
+ * 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 <utility>
+
+#include "hbase/client/async-batch-rpc-retrying-caller.h"
+#include "hbase/client/raw-async-table.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+
+using hbase::security::User;
+
+namespace hbase {
+
+template <typename RESP>
+std::shared_ptr<SingleRequestCallerBuilder<RESP>> RawAsyncTable::CreateCallerBuilder(
+    std::string row, std::chrono::nanoseconds rpc_timeout) {
+  return connection_->caller_factory()
+      ->Single<RESP>()
+      ->table(table_name_)
+      ->row(row)
+      ->rpc_timeout(rpc_timeout)
+      ->operation_timeout(connection_conf_->operation_timeout())
+      ->pause(connection_conf_->pause())
+      ->max_retries(connection_conf_->max_retries())
+      ->start_log_errors_count(connection_conf_->start_log_errors_count());
+}
+
+template <typename REQ, typename PREQ, typename PRESP, typename RESP>
+folly::Future<RESP> RawAsyncTable::Call(
+    std::shared_ptr<RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
+    std::shared_ptr<RegionLocation> loc, const REQ& req,
+    const ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
+    const RespConverter<RESP, PRESP> resp_converter) {
+  std::unique_ptr<PREQ> preq = req_converter(req, loc->region_name());
+
+  // No need to make take a callable argument, it is always the same
+  return rpc_client
+      ->AsyncCall(loc->server_name().host_name(), loc->server_name().port(), std::move(preq),
+                  User::defaultUser(), "ClientService")
+      .then(
+          [resp_converter](const std::unique_ptr<PRESP>& presp) { return resp_converter(*presp); });
+}
+
+folly::Future<std::shared_ptr<Result>> RawAsyncTable::Get(const hbase::Get& get) {
+  auto caller =
+      CreateCallerBuilder<std::shared_ptr<Result>>(get.row(), connection_conf_->read_rpc_timeout())
+          ->action([=, &get](std::shared_ptr<hbase::HBaseRpcController> controller,
+                             std::shared_ptr<hbase::RegionLocation> loc,
+                             std::shared_ptr<hbase::RpcClient> rpc_client)
+                       -> folly::Future<std::shared_ptr<hbase::Result>> {
+                         return Call<hbase::Get, hbase::Request, hbase::Response,
+                                     std::shared_ptr<hbase::Result>>(
+                             rpc_client, controller, loc, get,
+                             &hbase::RequestConverter::ToGetRequest,
+                             &hbase::ResponseConverter::FromGetResponse);
+                       })
+          ->Build();
+
+  // Return the Future we obtain from the call(). However, we do not want the Caller to go out of
+  // context and get deallocated since the caller injects a lot of closures which capture [this, &]
+  // which is use-after-free. We are just passing an identity closure capturing caller by value to
+  // ensure  that the lifecycle of the Caller object is longer than the retry lambdas.
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+folly::Future<std::shared_ptr<Result>> RawAsyncTable::Increment(const hbase::Increment& incr) {
+  auto caller =
+      CreateCallerBuilder<std::shared_ptr<Result>>(incr.row(),
+                                                   connection_conf_->write_rpc_timeout())
+          ->action([=, &incr](std::shared_ptr<hbase::HBaseRpcController> controller,
+                              std::shared_ptr<hbase::RegionLocation> loc,
+                              std::shared_ptr<hbase::RpcClient>
+                                  rpc_client) -> folly::Future<std::shared_ptr<Result>> {
+            return Call<hbase::Increment, hbase::Request, hbase::Response, std::shared_ptr<Result>>(
+                rpc_client, controller, loc, incr,
+                &hbase::RequestConverter::IncrementToMutateRequest,
+                &hbase::ResponseConverter::FromMutateResponse);
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<folly::Unit> RawAsyncTable::Put(const hbase::Put& put) {
+  auto caller =
+      CreateCallerBuilder<folly::Unit>(put.row(), connection_conf_->write_rpc_timeout())
+          ->action([=, &put](
+                       std::shared_ptr<hbase::HBaseRpcController> controller,
+                       std::shared_ptr<hbase::RegionLocation> loc,
+                       std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<folly::Unit> {
+            return Call<hbase::Put, hbase::Request, hbase::Response, folly::Unit>(
+                rpc_client, controller, loc, put, &hbase::RequestConverter::ToMutateRequest,
+                [](const Response& r) -> folly::Unit { return folly::unit; });
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<bool> RawAsyncTable::CheckAndPut(const std::string& row, const std::string& family,
+                                               const std::string& qualifier,
+                                               const std::string& value, const hbase::Put& put,
+                                               const pb::CompareType& compare_op) {
+  auto caller =
+      CreateCallerBuilder<bool>(row, connection_conf_->write_rpc_timeout())
+          ->action([=, &put](std::shared_ptr<hbase::HBaseRpcController> controller,
+                             std::shared_ptr<hbase::RegionLocation> loc,
+                             std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<bool> {
+            return Call<hbase::Put, hbase::Request, hbase::Response, bool>(
+                rpc_client, controller, loc, put,
+                // request conversion
+                [=, &put](const hbase::Put& put,
+                          const std::string& region_name) -> std::unique_ptr<Request> {
+                  auto checkReq = RequestConverter::CheckAndPutToMutateRequest(
+                      row, family, qualifier, value, compare_op, put, region_name);
+                  return checkReq;
+                },
+                // response conversion
+                &ResponseConverter::BoolFromMutateResponse);
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<bool> RawAsyncTable::CheckAndDelete(const std::string& row, const std::string& family,
+                                                  const std::string& qualifier,
+                                                  const std::string& value,
+                                                  const hbase::Delete& del,
+                                                  const pb::CompareType& compare_op) {
+  auto caller =
+      CreateCallerBuilder<bool>(row, connection_conf_->write_rpc_timeout())
+          ->action([=, &del](std::shared_ptr<hbase::HBaseRpcController> controller,
+                             std::shared_ptr<hbase::RegionLocation> loc,
+                             std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<bool> {
+            return Call<hbase::Delete, hbase::Request, hbase::Response, bool>(
+                rpc_client, controller, loc, del,
+                // request conversion
+                [=, &del](const hbase::Delete& del,
+                          const std::string& region_name) -> std::unique_ptr<Request> {
+                  auto checkReq = RequestConverter::CheckAndDeleteToMutateRequest(
+                      row, family, qualifier, value, compare_op, del, region_name);
+                  return checkReq;
+                },
+                // response conversion
+                &ResponseConverter::BoolFromMutateResponse);
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<folly::Unit> RawAsyncTable::Delete(const hbase::Delete& del) {
+  auto caller =
+      CreateCallerBuilder<folly::Unit>(del.row(), connection_conf_->write_rpc_timeout())
+          ->action([=, &del](
+                       std::shared_ptr<hbase::HBaseRpcController> controller,
+                       std::shared_ptr<hbase::RegionLocation> loc,
+                       std::shared_ptr<hbase::RpcClient> rpc_client) -> folly::Future<folly::Unit> {
+            return Call<hbase::Delete, hbase::Request, hbase::Response, folly::Unit>(
+                rpc_client, controller, loc, del, &hbase::RequestConverter::DeleteToMutateRequest,
+                [](const Response& r) -> folly::Unit { return folly::unit; });
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<std::shared_ptr<Result>> RawAsyncTable::Append(const hbase::Append& append) {
+  auto caller =
+      CreateCallerBuilder<std::shared_ptr<Result>>(append.row(),
+                                                   connection_conf_->write_rpc_timeout())
+          ->action([=, &append](std::shared_ptr<hbase::HBaseRpcController> controller,
+                                std::shared_ptr<hbase::RegionLocation> loc,
+                                std::shared_ptr<hbase::RpcClient>
+                                    rpc_client) -> folly::Future<std::shared_ptr<Result>> {
+            return Call<hbase::Append, hbase::Request, hbase::Response, std::shared_ptr<Result>>(
+                rpc_client, controller, loc, append,
+                &hbase::RequestConverter::AppendToMutateRequest,
+                &hbase::ResponseConverter::FromMutateResponse);
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
+folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> RawAsyncTable::Get(
+    const std::vector<hbase::Get>& gets) {
+  std::vector<std::shared_ptr<hbase::Row>> rows;
+  for (auto get : gets) {
+    std::shared_ptr<hbase::Row> srow = std::make_shared<hbase::Get>(get);
+    rows.push_back(srow);
+  }
+  return this->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(
+      rows, connection_conf_->read_rpc_timeout());
+}
+
+template <typename REQ, typename RESP>
+folly::Future<std::vector<folly::Try<RESP>>> RawAsyncTable::Batch(
+    const std::vector<REQ>& rows, std::chrono::nanoseconds timeout) {
+  auto caller = connection_->caller_factory()
+                    ->Batch<REQ, RESP>()
+                    ->table(table_name_)
+                    ->actions(std::make_shared<std::vector<REQ>>(rows))
+                    ->rpc_timeout(timeout)
+                    ->operation_timeout(connection_conf_->operation_timeout())
+                    ->pause(connection_conf_->pause())
+                    ->max_attempts(connection_conf_->max_retries())
+                    ->start_log_errors_count(connection_conf_->start_log_errors_count())
+                    ->Build();
+
+  return caller->Call().then([caller](auto r) { return r; });
+}
+
+void RawAsyncTable::Scan(const hbase::Scan& scan, std::shared_ptr<RawScanResultConsumer> consumer) {
+  auto scanner = AsyncClientScanner::Create(
+      connection_, SetDefaultScanConfig(scan), table_name_, consumer, connection_conf_->pause(),
+      connection_conf_->max_retries(), connection_conf_->scan_timeout(),
+      connection_conf_->rpc_timeout(), connection_conf_->start_log_errors_count());
+  scanner->Start();
+}
+
+std::shared_ptr<hbase::Scan> RawAsyncTable::SetDefaultScanConfig(const hbase::Scan& scan) {
+  // always create a new scan object as we may reset the start row later.
+  auto new_scan = std::make_shared<hbase::Scan>(scan);
+  if (new_scan->Caching() <= 0) {
+    new_scan->SetCaching(default_scanner_caching_);
+  }
+  if (new_scan->MaxResultSize() <= 0) {
+    new_scan->SetMaxResultSize(default_scanner_max_result_size_);
+  }
+  return new_scan;
+}
+
+folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> RawAsyncTable::Put(
+    const std::vector<hbase::Put>& puts) {
+  std::vector<std::shared_ptr<hbase::Row>> rows;
+  for (auto put : puts) {
+    std::shared_ptr<hbase::Row> srow = std::make_shared<hbase::Put>(put);
+    rows.push_back(srow);
+  }
+  return this->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(
+      rows, connection_conf_->write_rpc_timeout());
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/region-result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/region-result.cc b/hbase-native-client/src/hbase/client/region-result.cc
new file mode 100644
index 0000000..28c4861
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/region-result.cc
@@ -0,0 +1,53 @@
+/*
+ * 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 "hbase/client/region-result.h"
+#include <glog/logging.h>
+#include <stdexcept>
+
+using hbase::pb::RegionLoadStats;
+
+namespace hbase {
+
+RegionResult::RegionResult() {}
+
+RegionResult::~RegionResult() {}
+
+void RegionResult::AddResultOrException(int32_t index, std::shared_ptr<hbase::Result> result,
+                                        std::shared_ptr<folly::exception_wrapper> exc) {
+  auto index_found = result_or_excption_.find(index);
+  if (index_found == result_or_excption_.end()) {
+    result_or_excption_[index] = std::make_tuple(result ? result : nullptr, exc ? exc : nullptr);
+  } else {
+    throw std::runtime_error("Index " + std::to_string(index) +
+                             " already set with ResultOrException");
+  }
+}
+
+void RegionResult::set_stat(std::shared_ptr<RegionLoadStats> stat) { stat_ = stat; }
+
+int RegionResult::ResultOrExceptionSize() const { return result_or_excption_.size(); }
+
+std::shared_ptr<ResultOrExceptionTuple> RegionResult::ResultOrException(int32_t index) const {
+  return std::make_shared<ResultOrExceptionTuple>(result_or_excption_.at(index));
+}
+
+const std::shared_ptr<RegionLoadStats>& RegionResult::stat() const { return stat_; }
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/request-converter-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/request-converter-test.cc b/hbase-native-client/src/hbase/client/request-converter-test.cc
new file mode 100644
index 0000000..0878519
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/request-converter-test.cc
@@ -0,0 +1,126 @@
+/*
+ * 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 "hbase/client/request-converter.h"
+
+#include <gtest/gtest.h>
+#include <limits>
+#include "hbase/connection/request.h"
+#include "hbase/client/get.h"
+#include "hbase/client/scan.h"
+
+using hbase::Get;
+using hbase::Scan;
+
+using hbase::pb::GetRequest;
+using hbase::pb::RegionSpecifier;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+using hbase::pb::ScanRequest;
+
+TEST(RequestConverter, ToGet) {
+  std::string row_str = "row-test";
+  Get get(row_str);
+  get.AddFamily("family-1");
+  get.AddFamily("family-2");
+  get.AddFamily("family-3");
+  get.AddColumn("family-2", "qualifier-1");
+  get.AddColumn("family-2", "qualifier-2");
+  get.AddColumn("family-2", "qualifier-3");
+  get.SetCacheBlocks(false);
+  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  get.SetMaxVersions(2);
+  get.SetTimeRange(10000, 20000);
+  std::string region_name("RegionName");
+
+  auto req = hbase::RequestConverter::ToGetRequest(get, region_name);
+  auto msg = std::static_pointer_cast<GetRequest>(req->req_msg());
+
+  // Tests whether the PB object is properly set or not.
+  ASSERT_TRUE(msg->has_region());
+  ASSERT_TRUE(msg->region().has_value());
+  EXPECT_EQ(msg->region().value(), region_name);
+
+  ASSERT_TRUE(msg->has_get());
+  EXPECT_EQ(msg->get().row(), row_str);
+  EXPECT_FALSE(msg->get().cache_blocks());
+  EXPECT_EQ(msg->get().consistency(), hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(msg->get().max_versions(), 2);
+  EXPECT_EQ(msg->get().column_size(), 3);
+  for (int i = 0; i < msg->get().column_size(); ++i) {
+    EXPECT_EQ(msg->get().column(i).family(), "family-" + std::to_string(i + 1));
+    for (int j = 0; j < msg->get().column(i).qualifier_size(); ++j) {
+      EXPECT_EQ(msg->get().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
+    }
+  }
+}
+
+TEST(RequestConverter, ToScan) {
+  std::string start_row("start-row");
+  std::string stop_row("stop-row");
+  hbase::Scan scan;
+  scan.AddFamily("family-1");
+  scan.AddFamily("family-2");
+  scan.AddFamily("family-3");
+  scan.AddColumn("family-2", "qualifier-1");
+  scan.AddColumn("family-2", "qualifier-2");
+  scan.AddColumn("family-2", "qualifier-3");
+  scan.SetReversed(true);
+  scan.SetStartRow(start_row);
+  scan.SetStopRow(stop_row);
+  scan.SetCaching(3);
+  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  scan.SetCacheBlocks(true);
+  scan.SetAllowPartialResults(true);
+  scan.SetLoadColumnFamiliesOnDemand(true);
+  scan.SetMaxVersions(5);
+  scan.SetTimeRange(10000, 20000);
+  std::string region_name("RegionName");
+
+  auto req = hbase::RequestConverter::ToScanRequest(scan, region_name);
+  auto msg = std::static_pointer_cast<ScanRequest>(req->req_msg());
+
+  // Tests whether the PB object is properly set or not.
+  ASSERT_TRUE(msg->has_region());
+  ASSERT_TRUE(msg->region().has_value());
+  EXPECT_EQ(msg->region().value(), region_name);
+
+  ASSERT_TRUE(msg->has_scan());
+  EXPECT_TRUE(msg->scan().reversed());
+  EXPECT_EQ(msg->scan().start_row(), start_row);
+  EXPECT_EQ(msg->scan().stop_row(), stop_row);
+  EXPECT_FALSE(msg->scan().small());
+  EXPECT_EQ(msg->scan().caching(), 3);
+  EXPECT_EQ(msg->scan().consistency(), hbase::pb::Consistency::TIMELINE);
+  EXPECT_TRUE(msg->scan().cache_blocks());
+  EXPECT_TRUE(msg->scan().allow_partial_results());
+  EXPECT_TRUE(msg->scan().load_column_families_on_demand());
+  EXPECT_EQ(msg->scan().max_versions(), 5);
+  EXPECT_EQ(msg->scan().max_result_size(), std::numeric_limits<uint64_t>::max());
+
+  EXPECT_EQ(msg->scan().column_size(), 3);
+  for (int i = 0; i < msg->scan().column_size(); ++i) {
+    EXPECT_EQ(msg->scan().column(i).family(), "family-" + std::to_string(i + 1));
+    for (int j = 0; j < msg->scan().column(i).qualifier_size(); ++j) {
+      EXPECT_EQ(msg->scan().column(i).qualifier(j), "qualifier-" + std::to_string(j + 1));
+    }
+  }
+  ASSERT_FALSE(msg->client_handles_partials());
+  ASSERT_FALSE(msg->client_handles_heartbeats());
+  ASSERT_FALSE(msg->track_scan_metrics());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/request-converter.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/request-converter.cc b/hbase-native-client/src/hbase/client/request-converter.cc
new file mode 100644
index 0000000..a57ac31
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/request-converter.cc
@@ -0,0 +1,368 @@
+/*
+ * 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 "hbase/client/request-converter.h"
+
+#include <folly/Conv.h>
+
+#include <utility>
+#include "hbase/if/Client.pb.h"
+
+using hbase::pb::GetRequest;
+using hbase::pb::MutationProto;
+using hbase::pb::RegionAction;
+using hbase::pb::RegionSpecifier;
+using hbase::pb::RegionSpecifier_RegionSpecifierType;
+using hbase::pb::ScanRequest;
+
+namespace hbase {
+
+RequestConverter::~RequestConverter() {}
+
+RequestConverter::RequestConverter() {}
+
+void RequestConverter::SetRegion(const std::string &region_name,
+                                 RegionSpecifier *region_specifier) {
+  region_specifier->set_type(
+      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_REGION_NAME);
+  region_specifier->set_value(region_name);
+}
+
+std::unique_ptr<Request> RequestConverter::ToGetRequest(const Get &get,
+                                                        const std::string &region_name) {
+  auto pb_req = Request::get();
+  auto pb_msg = std::static_pointer_cast<GetRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+  pb_msg->set_allocated_get((RequestConverter::ToGet(get)).release());
+  return pb_req;
+}
+
+std::unique_ptr<hbase::pb::Scan> RequestConverter::ToScan(const Scan &scan) {
+  auto pb_scan = std::make_unique<hbase::pb::Scan>();
+  pb_scan->set_max_versions(scan.MaxVersions());
+  pb_scan->set_cache_blocks(scan.CacheBlocks());
+  pb_scan->set_reversed(scan.IsReversed());
+  pb_scan->set_caching(scan.Caching());
+  pb_scan->set_start_row(scan.StartRow());
+  pb_scan->set_stop_row(scan.StopRow());
+  pb_scan->set_consistency(scan.Consistency());
+  pb_scan->set_max_result_size(scan.MaxResultSize());
+  pb_scan->set_allow_partial_results(scan.AllowPartialResults());
+  pb_scan->set_load_column_families_on_demand(scan.LoadColumnFamiliesOnDemand());
+
+  if (!scan.Timerange().IsAllTime()) {
+    hbase::pb::TimeRange *pb_time_range = pb_scan->mutable_time_range();
+    pb_time_range->set_from(scan.Timerange().MinTimeStamp());
+    pb_time_range->set_to(scan.Timerange().MaxTimeStamp());
+  }
+
+  if (scan.HasFamilies()) {
+    for (const auto &family : scan.FamilyMap()) {
+      auto column = pb_scan->add_column();
+      column->set_family(family.first);
+      for (const auto &qualifier : family.second) {
+        column->add_qualifier(qualifier);
+      }
+    }
+  }
+
+  if (scan.filter() != nullptr) {
+    pb_scan->set_allocated_filter(Filter::ToProto(*(scan.filter())).release());
+  }
+
+  return std::move(pb_scan);
+}
+
+std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
+                                                         const std::string &region_name) {
+  auto pb_req = Request::scan();
+  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
+
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_scan(ToScan(scan).release());
+
+  SetCommonScanRequestFields(pb_msg, false);
+
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
+                                                         const std::string &region_name,
+                                                         int32_t num_rows, bool close_scanner) {
+  auto pb_req = Request::scan();
+  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
+
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_scan(ToScan(scan).release());
+
+  pb_msg->set_number_of_rows(num_rows);
+  pb_msg->set_close_scanner(close_scanner);
+
+  SetCommonScanRequestFields(pb_msg, false);
+
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::ToScanRequest(int64_t scanner_id, int32_t num_rows,
+                                                         bool close_scanner) {
+  auto pb_req = Request::scan();
+  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
+
+  pb_msg->set_number_of_rows(num_rows);
+  pb_msg->set_close_scanner(close_scanner);
+  pb_msg->set_scanner_id(scanner_id);
+
+  SetCommonScanRequestFields(pb_msg, false);
+
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::ToScanRequest(int64_t scanner_id, int32_t num_rows,
+                                                         bool close_scanner,
+                                                         int64_t next_call_seq_id, bool renew) {
+  auto pb_req = Request::scan();
+  auto pb_msg = std::static_pointer_cast<ScanRequest>(pb_req->req_msg());
+
+  pb_msg->set_number_of_rows(num_rows);
+  pb_msg->set_close_scanner(close_scanner);
+  pb_msg->set_scanner_id(scanner_id);
+  pb_msg->set_next_call_seq(next_call_seq_id);
+
+  SetCommonScanRequestFields(pb_msg, renew);
+  return pb_req;
+}
+
+void RequestConverter::SetCommonScanRequestFields(std::shared_ptr<hbase::pb::ScanRequest> pb_msg,
+                                                  bool renew) {
+  // TODO We will change these later when we implement partial results and heartbeats, etc
+  pb_msg->set_client_handles_partials(false);
+  pb_msg->set_client_handles_heartbeats(false);
+  pb_msg->set_track_scan_metrics(false);
+  pb_msg->set_renew(renew);
+  // TODO: set scan limit
+}
+
+std::unique_ptr<Request> RequestConverter::ToMultiRequest(
+    const ActionsByRegion &actions_by_region) {
+  auto pb_req = Request::multi();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MultiRequest>(pb_req->req_msg());
+
+  for (const auto &action_by_region : actions_by_region) {
+    auto pb_region_action = pb_msg->add_regionaction();
+    RequestConverter::SetRegion(action_by_region.first, pb_region_action->mutable_region());
+    int action_num = 0;
+    for (const auto &region_action : action_by_region.second->actions()) {
+      auto pb_action = pb_region_action->add_action();
+      auto pget = region_action->action();
+      // We store only hbase::Get in hbase::Action as of now. It will be changed later on.
+      CHECK(pget) << "Unexpected. action can't be null.";
+      std::string error_msg("");
+      if (typeid(*pget) == typeid(hbase::Get)) {
+        auto getp = dynamic_cast<hbase::Get *>(pget.get());
+        pb_action->set_allocated_get(RequestConverter::ToGet(*getp).release());
+      } else if (typeid(*pget) == typeid(hbase::Put)) {
+        auto putp = dynamic_cast<hbase::Put *>(pget.get());
+        pb_action->set_allocated_mutation(
+            RequestConverter::ToMutation(MutationType::MutationProto_MutationType_PUT, *putp, -1)
+                .release());
+      } else {
+        throw std::runtime_error("Unexpected action type encountered.");
+      }
+      pb_action->set_index(action_num);
+      action_num++;
+    }
+  }
+  return pb_req;
+}
+
+std::unique_ptr<hbase::pb::Get> RequestConverter::ToGet(const Get &get) {
+  auto pb_get = std::make_unique<hbase::pb::Get>();
+  pb_get->set_max_versions(get.MaxVersions());
+  pb_get->set_cache_blocks(get.CacheBlocks());
+  pb_get->set_consistency(get.Consistency());
+
+  if (!get.Timerange().IsAllTime()) {
+    hbase::pb::TimeRange *pb_time_range = pb_get->mutable_time_range();
+    pb_time_range->set_from(get.Timerange().MinTimeStamp());
+    pb_time_range->set_to(get.Timerange().MaxTimeStamp());
+  }
+  pb_get->set_row(get.row());
+  if (get.HasFamilies()) {
+    for (const auto &family : get.FamilyMap()) {
+      auto column = pb_get->add_column();
+      column->set_family(family.first);
+      for (const auto &qualifier : family.second) {
+        column->add_qualifier(qualifier);
+      }
+    }
+  }
+
+  if (get.filter() != nullptr) {
+    pb_get->set_allocated_filter(Filter::ToProto(*(get.filter())).release());
+  }
+  return pb_get;
+}
+
+std::unique_ptr<MutationProto> RequestConverter::ToMutation(const MutationType type,
+                                                            const Mutation &mutation,
+                                                            const int64_t nonce) {
+  auto pb_mut = std::make_unique<MutationProto>();
+  pb_mut->set_row(mutation.row());
+  pb_mut->set_mutate_type(type);
+  pb_mut->set_durability(mutation.Durability());
+  pb_mut->set_timestamp(mutation.TimeStamp());
+  // TODO: set attributes from the mutation (key value pairs).
+
+  if (nonce > 0) {
+    pb_mut->set_nonce(nonce);
+  }
+
+  for (const auto &family : mutation.FamilyMap()) {
+    for (const auto &cell : family.second) {
+      auto column = pb_mut->add_column_value();
+      column->set_family(cell->Family());
+      auto qual = column->add_qualifier_value();
+      qual->set_qualifier(cell->Qualifier());
+      qual->set_timestamp(cell->Timestamp());
+      auto cell_type = cell->Type();
+      if (type == pb::MutationProto_MutationType_DELETE ||
+          (type == pb::MutationProto_MutationType_PUT && IsDelete(cell_type))) {
+        qual->set_delete_type(ToDeleteType(cell_type));
+      }
+
+      qual->set_value(cell->Value());
+    }
+  }
+  return std::move(pb_mut);
+}
+
+DeleteType RequestConverter::ToDeleteType(const CellType type) {
+  switch (type) {
+    case CellType::DELETE:
+      return pb::MutationProto_DeleteType_DELETE_ONE_VERSION;
+    case CellType::DELETE_COLUMN:
+      return pb::MutationProto_DeleteType_DELETE_MULTIPLE_VERSIONS;
+    case CellType::DELETE_FAMILY:
+      return pb::MutationProto_DeleteType_DELETE_FAMILY;
+    case CellType::DELETE_FAMILY_VERSION:
+      return pb::MutationProto_DeleteType_DELETE_FAMILY_VERSION;
+    default:
+      throw std::runtime_error("Unknown delete type: " + folly::to<std::string>(type));
+  }
+}
+
+bool RequestConverter::IsDelete(const CellType type) {
+  return CellType::DELETE <= type && type <= CellType::DELETE_FAMILY;
+}
+
+std::unique_ptr<Request> RequestConverter::ToMutateRequest(const Put &put,
+                                                           const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_PUT, put, -1).release());
+
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::CheckAndPutToMutateRequest(
+    const std::string &row, const std::string &family, const std::string &qualifier,
+    const std::string &value, const pb::CompareType compare_op, const hbase::Put &put,
+    const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_PUT, put, -1).release());
+  ::hbase::pb::Condition *cond = pb_msg->mutable_condition();
+  cond->set_row(row);
+  cond->set_family(family);
+  cond->set_qualifier(qualifier);
+  cond->set_allocated_comparator(
+      Comparator::ToProto(*(ComparatorFactory::BinaryComparator(value).get())).release());
+  cond->set_compare_type(compare_op);
+
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::CheckAndDeleteToMutateRequest(
+    const std::string &row, const std::string &family, const std::string &qualifier,
+    const std::string &value, const pb::CompareType compare_op, const hbase::Delete &del,
+    const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_DELETE, del, -1).release());
+  ::hbase::pb::Condition *cond = pb_msg->mutable_condition();
+  cond->set_row(row);
+  cond->set_family(family);
+  cond->set_qualifier(qualifier);
+  cond->set_allocated_comparator(
+      Comparator::ToProto(*(ComparatorFactory::BinaryComparator(value).get())).release());
+  cond->set_compare_type(compare_op);
+
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::DeleteToMutateRequest(const Delete &del,
+                                                                 const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_DELETE, del, -1).release());
+
+  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
+  return pb_req;
+}
+std::unique_ptr<Request> RequestConverter::IncrementToMutateRequest(
+    const Increment &incr, const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_INCREMENT, incr, -1).release());
+
+  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
+  return pb_req;
+}
+
+std::unique_ptr<Request> RequestConverter::AppendToMutateRequest(const Append &append,
+                                                                 const std::string &region_name) {
+  auto pb_req = Request::mutate();
+  auto pb_msg = std::static_pointer_cast<hbase::pb::MutateRequest>(pb_req->req_msg());
+  RequestConverter::SetRegion(region_name, pb_msg->mutable_region());
+
+  pb_msg->set_allocated_mutation(
+      ToMutation(MutationType::MutationProto_MutationType_APPEND, append, -1).release());
+
+  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
+  return pb_req;
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/response-converter.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/response-converter.cc b/hbase-native-client/src/hbase/client/response-converter.cc
new file mode 100644
index 0000000..f3b78fd
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/response-converter.cc
@@ -0,0 +1,221 @@
+/*
+ * 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 "hbase/client/response-converter.h"
+#include <glog/logging.h>
+#include <stdexcept>
+#include <string>
+#include <utility>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/multi-response.h"
+#include "hbase/exceptions/exception.h"
+
+using hbase::pb::GetResponse;
+using hbase::pb::MutateResponse;
+using hbase::pb::ScanResponse;
+using hbase::pb::RegionLoadStats;
+
+namespace hbase {
+
+ResponseConverter::ResponseConverter() {}
+
+ResponseConverter::~ResponseConverter() {}
+
+// impl note: we are returning shared_ptr's instead of unique_ptr's because these
+// go inside folly::Future's, making the move semantics extremely tricky.
+std::shared_ptr<Result> ResponseConverter::FromGetResponse(const Response& resp) {
+  auto get_resp = std::static_pointer_cast<GetResponse>(resp.resp_msg());
+  VLOG(3) << "FromGetResponse:" << get_resp->ShortDebugString();
+  return ToResult(get_resp->result(), resp.cell_scanner());
+}
+
+std::shared_ptr<Result> ResponseConverter::FromMutateResponse(const Response& resp) {
+  auto mutate_resp = std::static_pointer_cast<MutateResponse>(resp.resp_msg());
+  hbase::pb::Result result = mutate_resp->result();
+  return ToResult(mutate_resp->result(), resp.cell_scanner());
+}
+
+bool ResponseConverter::BoolFromMutateResponse(const Response& resp) {
+  auto mutate_resp = std::static_pointer_cast<MutateResponse>(resp.resp_msg());
+  return mutate_resp->processed();
+}
+
+std::shared_ptr<Result> ResponseConverter::ToResult(
+    const hbase::pb::Result& result, const std::shared_ptr<CellScanner> cell_scanner) {
+  std::vector<std::shared_ptr<Cell>> vcells;
+  for (auto cell : result.cell()) {
+    std::shared_ptr<Cell> pcell =
+        std::make_shared<Cell>(cell.row(), cell.family(), cell.qualifier(), cell.timestamp(),
+                               cell.value(), static_cast<hbase::CellType>(cell.cell_type()));
+    vcells.push_back(pcell);
+  }
+
+  // iterate over the cells coming from rpc codec
+  if (cell_scanner != nullptr) {
+    int cells_read = 0;
+    while (cells_read != result.associated_cell_count()) {
+      if (cell_scanner->Advance()) {
+        vcells.push_back(cell_scanner->Current());
+        cells_read += 1;
+      } else {
+        LOG(ERROR) << "CellScanner::Advance() returned false unexpectedly. Cells Read:- "
+                   << cells_read << "; Expected Cell Count:- " << result.associated_cell_count();
+        std::runtime_error("CellScanner::Advance() returned false unexpectedly");
+      }
+    }
+  }
+  return std::make_shared<Result>(vcells, result.exists(), result.stale(), result.partial());
+}
+
+std::vector<std::shared_ptr<Result>> ResponseConverter::FromScanResponse(const Response& resp) {
+  auto scan_resp = std::static_pointer_cast<ScanResponse>(resp.resp_msg());
+  return FromScanResponse(scan_resp, resp.cell_scanner());
+}
+
+std::vector<std::shared_ptr<Result>> ResponseConverter::FromScanResponse(
+    const std::shared_ptr<ScanResponse> scan_resp, std::shared_ptr<CellScanner> cell_scanner) {
+  VLOG(3) << "FromScanResponse:" << scan_resp->ShortDebugString()
+          << " cell_scanner:" << (cell_scanner != nullptr);
+  int num_results =
+      cell_scanner != nullptr ? scan_resp->cells_per_result_size() : scan_resp->results_size();
+
+  std::vector<std::shared_ptr<Result>> results{static_cast<size_t>(num_results)};
+  for (int i = 0; i < num_results; i++) {
+    if (cell_scanner != nullptr) {
+      // Cells are out in cellblocks.  Group them up again as Results.  How many to read at a
+      // time will be found in getCellsLength -- length here is how many Cells in the i'th Result
+      int num_cells = scan_resp->cells_per_result(i);
+
+      std::vector<std::shared_ptr<Cell>> vcells;
+      for (int j = 0; j < num_cells; j++) {
+        if (!cell_scanner->Advance()) {
+          std::string msg = "Results sent from server=" + std::to_string(num_results) +
+                            ". But only got " + std::to_string(i) +
+                            " results completely at client. Resetting the scanner to scan again.";
+          LOG(ERROR) << msg;
+          throw std::runtime_error(msg);
+        }
+        vcells.push_back(cell_scanner->Current());
+      }
+      // TODO: handle partial results per Result by checking partial_flag_per_result
+      results[i] = std::make_shared<Result>(vcells, false, scan_resp->stale(), false);
+    } else {
+      results[i] = ToResult(scan_resp->results(i), cell_scanner);
+    }
+  }
+
+  return results;
+}
+
+std::unique_ptr<hbase::MultiResponse> ResponseConverter::GetResults(
+    std::shared_ptr<Request> req, const Response& resp,
+    const ServerRequest::ActionsByRegion& actions_by_region) {
+  auto multi_req = std::static_pointer_cast<hbase::pb::MultiRequest>(req->req_msg());
+  auto multi_resp = std::static_pointer_cast<hbase::pb::MultiResponse>(resp.resp_msg());
+  VLOG(3) << "GetResults:" << multi_resp->ShortDebugString();
+  int req_region_action_count = multi_req->regionaction_size();
+  int res_region_action_count = multi_resp->regionactionresult_size();
+  if (req_region_action_count != res_region_action_count) {
+    throw std::runtime_error("Request mutation count=" + std::to_string(req_region_action_count) +
+                             " does not match response mutation result count=" +
+                             std::to_string(res_region_action_count));
+  }
+  auto multi_response = std::make_unique<hbase::MultiResponse>();
+  for (int32_t num = 0; num < res_region_action_count; num++) {
+    hbase::pb::RegionAction actions = multi_req->regionaction(num);
+    hbase::pb::RegionActionResult action_result = multi_resp->regionactionresult(num);
+    hbase::pb::RegionSpecifier rs = actions.region();
+    if (rs.has_type() && rs.type() != hbase::pb::RegionSpecifier::REGION_NAME) {
+      throw std::runtime_error("We support only encoded types for protobuf multi response.");
+    }
+
+    auto region_name = rs.value();
+    if (action_result.has_exception()) {
+      auto ew = ResponseConverter::GetRemoteException(action_result.exception());
+      VLOG(8) << "Store Remote Region Exception:- " << ew->what().toStdString() << "; Region["
+              << region_name << "];";
+      multi_response->AddRegionException(region_name, ew);
+      continue;
+    }
+
+    if (actions.action_size() != action_result.resultorexception_size()) {
+      throw std::runtime_error("actions.action_size=" + std::to_string(actions.action_size()) +
+                               ", action_result.resultorexception_size=" +
+                               std::to_string(action_result.resultorexception_size()) +
+                               " for region " + actions.region().value());
+    }
+
+    auto multi_actions = actions_by_region.at(region_name)->actions();
+    uint64_t multi_actions_num = 0;
+    for (hbase::pb::ResultOrException roe : action_result.resultorexception()) {
+      std::shared_ptr<Result> result;
+      std::shared_ptr<folly::exception_wrapper> ew;
+      if (roe.has_exception()) {
+        auto ew = ResponseConverter::GetRemoteException(roe.exception());
+        VLOG(8) << "Store Remote Region Exception:- " << ew->what().toStdString() << "; Region["
+                << region_name << "];";
+        multi_response->AddRegionException(region_name, ew);
+      } else if (roe.has_result()) {
+        result = ToResult(roe.result(), resp.cell_scanner());
+      } else if (roe.has_service_result()) {
+        // TODO Not processing Coprocessor Service Result;
+      } else {
+        // Sometimes, the response is just "it was processed". Generally, this occurs for things
+        // like mutateRows where either we get back 'processed' (or not) and optionally some
+        // statistics about the regions we touched.
+        std::vector<std::shared_ptr<Cell>> empty_cells;
+        result = std::make_shared<Result>(empty_cells, multi_resp->processed() ? true : false,
+                                          false, false);
+      }
+      // We add the original index of the multi-action so that when populating the response back we
+      // do it as per the action index
+      multi_response->AddRegionResult(
+          region_name, multi_actions[multi_actions_num]->original_index(), std::move(result), ew);
+      multi_actions_num++;
+    }
+  }
+
+  if (multi_resp->has_regionstatistics()) {
+    hbase::pb::MultiRegionLoadStats stats = multi_resp->regionstatistics();
+    for (int i = 0; i < stats.region_size(); i++) {
+      multi_response->AddStatistic(stats.region(i).value(),
+                                   std::make_shared<RegionLoadStats>(stats.stat(i)));
+    }
+  }
+  return multi_response;
+}
+
+std::shared_ptr<folly::exception_wrapper> ResponseConverter::GetRemoteException(
+    const hbase::pb::NameBytesPair& exc_resp) {
+  std::string what;
+  std::string exception_class_name = exc_resp.has_name() ? exc_resp.name() : "";
+  std::string stack_trace = exc_resp.has_value() ? exc_resp.value() : "";
+
+  what.append(exception_class_name).append(stack_trace);
+  auto remote_exception = std::make_unique<RemoteException>(what);
+  remote_exception->set_exception_class_name(exception_class_name)
+      ->set_stack_trace(stack_trace)
+      ->set_hostname("")
+      ->set_port(0);
+
+  return std::make_shared<folly::exception_wrapper>(
+      folly::make_exception_wrapper<RemoteException>(*remote_exception));
+}
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/result-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/result-test.cc b/hbase-native-client/src/hbase/client/result-test.cc
new file mode 100644
index 0000000..684c08d
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/result-test.cc
@@ -0,0 +1,322 @@
+/*
+ * 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 <glog/logging.h>
+#include <gtest/gtest.h>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/client/cell.h"
+#include "hbase/client/result.h"
+
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Result;
+using std::experimental::nullopt;
+
+void PopulateCells(std::vector<std::shared_ptr<Cell> > &cells) {
+  // Populate some Results
+  // We assume that for a single Cell, the corresponding row, families and
+  // qualifiers are present.
+  // We have also considered different versions in the test for the same row.
+  std::string row = "row";
+  for (int i = 0; i < 10; i++) {
+    std::string family = "family-" + std::to_string(i);
+    std::string column = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+
+    switch (i) {
+      case 5: {
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482113040506, "value-5", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
+        break;
+      }
+      case 8: {
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482113040506, "value-8", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482110969958, "value-Y", CellType::PUT));
+        break;
+      }
+      case 9: {
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482113040506, "value-9", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482111803856, "value-X", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482110969958, "value-Y", CellType::PUT));
+        cells.push_back(
+            std::make_shared<Cell>(row, family, column, 1482110876075, "value-Z", CellType::PUT));
+        break;
+      }
+      default: {
+        cells.push_back(std::make_shared<Cell>(
+            row, family, column, std::numeric_limits<int64_t>::max(), value, CellType::PUT));
+      }
+    }
+  }
+  return;
+}
+
+TEST(Result, EmptyResult) {
+  std::vector<std::shared_ptr<Cell> > cells;
+  Result result(cells, true, false, false);
+  EXPECT_EQ(true, result.IsEmpty());
+  EXPECT_EQ(0, result.Size());
+}
+
+TEST(Result, FilledResult) {
+  std::vector<std::shared_ptr<Cell> > cells;
+  PopulateCells(cells);
+
+  Result result(cells, true, false, false);
+
+  EXPECT_EQ(false, result.IsEmpty());
+  EXPECT_EQ(16, result.Size());
+
+  // Get Latest Cell for the given family and qualifier.
+  auto latest_cell(result.ColumnLatestCell("family", "column"));
+  // Nothing of the above family/qualifier combo is present so it should be
+  // nullptr
+  ASSERT_FALSE(latest_cell.get());
+
+  // Try to get the latest cell for the given family and qualifier.
+  latest_cell = result.ColumnLatestCell("family-4", "column-4");
+  // Now shouldn't be a nullptr
+  ASSERT_TRUE(latest_cell.get());
+  // And Value must match too
+  EXPECT_EQ("value-4", latest_cell->Value());
+
+  // Value will be nullptr as no such family and qualifier is present
+  ASSERT_FALSE(result.Value("family-4", "qualifier"));
+  // Value will be present as family and qualifier is present
+  ASSERT_TRUE(result.Value("family-4", "column-4") != nullopt);
+  // Value should be present and match.
+  EXPECT_EQ(latest_cell->Value(), (*result.ColumnLatestCell("family-4", "column-4")).Value());
+  EXPECT_EQ("value-5", (*result.ColumnLatestCell("family-5", "column-5")).Value());
+  EXPECT_EQ("value-8", (*result.ColumnLatestCell("family-8", "column-8")).Value());
+  EXPECT_EQ("value-7", *result.Value("family-7", "column-7"));
+
+  // Get cells for the given family and qualifier
+  auto column_cells = result.ColumnCells("family", "column");
+  // Size should be 0
+  EXPECT_EQ(0, column_cells.size());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-0", "column-0");
+  EXPECT_EQ(1, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-5", "column-5");
+  EXPECT_EQ(2, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("value-5", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-8", "column-8");
+  EXPECT_EQ(3, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("row", column_cells[2]->Row());
+  EXPECT_EQ("value-8", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("value-Y", column_cells[2]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Size shouldn't be 0 and Row() and Value() must match
+  column_cells = result.ColumnCells("family-9", "column-9");
+  EXPECT_EQ(4, column_cells.size());
+  EXPECT_EQ("row", column_cells[0]->Row());
+  EXPECT_EQ("row", column_cells[1]->Row());
+  EXPECT_EQ("row", column_cells[2]->Row());
+  EXPECT_EQ("row", column_cells[3]->Row());
+  EXPECT_EQ("value-9", column_cells[0]->Value());
+  EXPECT_EQ("value-X", column_cells[1]->Value());
+  EXPECT_EQ("value-Y", column_cells[2]->Value());
+  EXPECT_EQ("value-Z", column_cells[3]->Value());
+  EXPECT_EQ("row", result.Row());
+
+  // Test all the Cell values
+  const auto &result_cells = result.Cells();
+  int i = 0, j = 0;
+  for (const auto &cell : result_cells) {
+    std::string row = "row";
+    std::string family = "family-" + std::to_string(i);
+    std::string column = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+    switch (j) {
+      case 6:
+      case 10:
+      case 13: {
+        EXPECT_EQ("value-X", cell->Value());
+        ++j;
+        continue;
+      }
+      case 11:
+      case 14: {
+        EXPECT_EQ("value-Y", cell->Value());
+        ++j;
+        continue;
+      }
+      case 15: {
+        EXPECT_EQ("value-Z", cell->Value());
+        ++j;
+        continue;
+      }
+    }
+    EXPECT_EQ(row, cell->Row());
+    EXPECT_EQ(family, cell->Family());
+    EXPECT_EQ(column, cell->Qualifier());
+    EXPECT_EQ(value, cell->Value());
+    ++i;
+    ++j;
+  }
+
+  auto result_map_tmp = result.Map();
+  result_map_tmp["testf"]["testq"][1] = "value";
+  EXPECT_EQ(11, result_map_tmp.size());
+
+  auto result_map = result.Map();
+  EXPECT_EQ(10, result_map.size());
+
+  i = 0;
+  for (auto family_map : result_map) {
+    std::string family = "family-" + std::to_string(i);
+    std::string qualifier = "column-" + std::to_string(i);
+    std::string value = "value-" + std::to_string(i);
+    EXPECT_EQ(family, family_map.first);
+    for (auto qualifier_map : family_map.second) {
+      EXPECT_EQ(qualifier, qualifier_map.first);
+      j = 0;
+      for (auto version_map : qualifier_map.second) {
+        switch (i) {
+          case 5: {
+            if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-5", version_map.second);
+            }
+            break;
+          }
+          case 8: {
+            if (2 == j) {
+              EXPECT_EQ(1482110969958, version_map.first);
+              EXPECT_EQ("value-Y", version_map.second);
+            } else if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-8", version_map.second);
+            }
+            break;
+          }
+          case 9: {
+            if (3 == j) {
+              EXPECT_EQ(1482110876075, version_map.first);
+              EXPECT_EQ("value-Z", version_map.second);
+            } else if (2 == j) {
+              EXPECT_EQ(1482110969958, version_map.first);
+              EXPECT_EQ("value-Y", version_map.second);
+            } else if (1 == j) {
+              EXPECT_EQ(1482111803856, version_map.first);
+              EXPECT_EQ("value-X", version_map.second);
+            } else if (0 == j) {
+              EXPECT_EQ(1482113040506, version_map.first);
+              EXPECT_EQ("value-9", version_map.second);
+            }
+            break;
+          }
+          default: {
+            EXPECT_EQ(std::numeric_limits<int64_t>::max(), version_map.first);
+            EXPECT_EQ(value, version_map.second);
+          }
+        }
+        ++j;
+      }
+    }
+    ++i;
+  }
+
+  auto family_map = result.FamilyMap("family-0");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-0", qual_val_map.first);
+    EXPECT_EQ("value-0", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-1");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-1", qual_val_map.first);
+    EXPECT_EQ("value-1", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-5");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-5", qual_val_map.first);
+    EXPECT_EQ("value-5", qual_val_map.second);
+  }
+
+  family_map = result.FamilyMap("family-9");
+  EXPECT_EQ(1, family_map.size());
+  i = 0;
+  for (auto qual_val_map : family_map) {
+    EXPECT_EQ("column-9", qual_val_map.first);
+    EXPECT_EQ("value-9", qual_val_map.second);
+  }
+}
+
+TEST(Result, ResultEstimatedSize) {
+  CellType cell_type = CellType::PUT;
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  std::vector<std::shared_ptr<Cell> > cells;
+  Result empty(cells, true, false, false);
+
+  EXPECT_EQ(empty.EstimatedSize(), sizeof(Result));
+
+  cells.push_back(std::make_shared<Cell>("a", "a", "", timestamp, "", cell_type));
+  Result result1(cells, true, false, false);
+  EXPECT_TRUE(result1.EstimatedSize() > empty.EstimatedSize());
+
+  cells.push_back(std::make_shared<Cell>("a", "a", "", timestamp, "", cell_type));
+  Result result2(cells, true, false, false);
+  EXPECT_TRUE(result2.EstimatedSize() > result1.EstimatedSize());
+
+  LOG(INFO) << empty.EstimatedSize();
+  LOG(INFO) << result1.EstimatedSize();
+  LOG(INFO) << result2.EstimatedSize();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/result.cc b/hbase-native-client/src/hbase/client/result.cc
new file mode 100644
index 0000000..a2f56aa
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/result.cc
@@ -0,0 +1,146 @@
+/*
+ * 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 "hbase/client/result.h"
+
+namespace hbase {
+
+Result::~Result() {}
+
+Result::Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists, bool stale,
+               bool partial)
+    : exists_(exists), stale_(stale), partial_(partial), cells_(cells) {
+  row_ = (cells_.size() == 0 ? "" : cells_[0]->Row());
+}
+
+Result::Result(const Result &result) {
+  exists_ = result.exists_;
+  stale_ = result.stale_;
+  partial_ = result.partial_;
+  row_ = result.row_;
+  if (!result.cells_.empty()) {
+    for (const auto &cell : result.cells_) {
+      cells_.push_back(cell);
+    }
+  }
+}
+
+const std::vector<std::shared_ptr<Cell> > &Result::Cells() const { return cells_; }
+
+std::vector<std::shared_ptr<Cell> > Result::ColumnCells(const std::string &family,
+                                                        const std::string &qualifier) const {
+  std::vector<std::shared_ptr<Cell> > column_cells;
+  // TODO implement a BinarySearch here ?
+  for (const auto &cell : cells_) {
+    if (cell->Family() == family && cell->Qualifier() == qualifier) {
+      column_cells.push_back(cell);
+    }
+  }
+  return column_cells;
+}
+
+const std::shared_ptr<Cell> Result::ColumnLatestCell(const std::string &family,
+                                                     const std::string &qualifier) const {
+  // TODO implement a BinarySearch here ?
+  for (const auto &cell : cells_) {
+    // We find the latest(first) occurrence of the Cell for a given column and
+    // qualifier and break
+    if (cell->Family() == family && cell->Qualifier() == qualifier) {
+      return cell;
+    }
+  }
+  return nullptr;
+}
+
+optional<std::string> Result::Value(const std::string &family, const std::string &qualifier) const {
+  std::shared_ptr<Cell> latest_cell(ColumnLatestCell(family, qualifier));
+  if (latest_cell.get()) {
+    return optional<std::string>(latest_cell->Value());
+  }
+  return optional<std::string>();
+}
+
+bool Result::IsEmpty() const { return cells_.empty(); }
+
+const std::string &Result::Row() const { return row_; }
+
+int Result::Size() const { return cells_.size(); }
+
+ResultMap Result::Map() const {
+  ResultMap result_map;
+  for (const auto &cell : cells_) {
+    result_map[cell->Family()][cell->Qualifier()][cell->Timestamp()] = cell->Value();
+  }
+  return result_map;
+}
+
+std::map<std::string, std::string> Result::FamilyMap(const std::string &family) const {
+  std::map<std::string, std::string> family_map;
+  if (!IsEmpty()) {
+    auto result_map = Map();
+    auto itr = result_map.find(family);
+    if (itr == result_map.end()) {
+      return family_map;
+    }
+
+    for (auto qitr = itr->second.begin(); qitr != itr->second.end(); ++qitr) {
+      for (auto vitr = qitr->second.begin(); vitr != qitr->second.end(); ++vitr) {
+        // We break after inserting the first value. Result.java takes only
+        // the first value
+        family_map[qitr->first] = vitr->second;
+        break;
+      }
+    }
+  }
+
+  return family_map;
+}
+
+std::string Result::DebugString() const {
+  std::string ret{"keyvalues="};
+  if (IsEmpty()) {
+    ret += "NONE";
+    return ret;
+  }
+  ret += "{";
+  bool is_first = true;
+  for (const auto &cell : cells_) {
+    if (is_first) {
+      is_first = false;
+    } else {
+      ret += ", ";
+    }
+    ret += cell->DebugString();
+  }
+  ret += "}";
+
+  return ret;
+}
+
+size_t Result::EstimatedSize() const {
+  size_t s = sizeof(Result);
+  s += row_.capacity();
+  for (const auto c : cells_) {
+    s += sizeof(std::shared_ptr<Cell>);
+    s + c->EstimatedSize();
+  }
+  return s;
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/scan-result-cache-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/scan-result-cache-test.cc b/hbase-native-client/src/hbase/client/scan-result-cache-test.cc
new file mode 100644
index 0000000..4c10a05
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/scan-result-cache-test.cc
@@ -0,0 +1,177 @@
+/*
+ * 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 <folly/Conv.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include <vector>
+
+#include "hbase/client/cell.h"
+#include "hbase/client/result.h"
+#include "hbase/client/scan-result-cache.h"
+
+using hbase::ScanResultCache;
+using hbase::Result;
+using hbase::Cell;
+using hbase::CellType;
+
+using ResultVector = std::vector<std::shared_ptr<Result>>;
+
+std::shared_ptr<Cell> CreateCell(const int32_t &key, const std::string &family,
+                                 const std::string &column) {
+  auto row = folly::to<std::string>(key);
+  return std::make_shared<Cell>(row, family, column, std::numeric_limits<int64_t>::max(), row,
+                                CellType::PUT);
+}
+
+std::shared_ptr<Result> CreateResult(std::shared_ptr<Cell> cell, bool partial) {
+  return std::make_shared<Result>(std::vector<std::shared_ptr<Cell>>{cell}, false, false, partial);
+}
+
+TEST(ScanResultCacheTest, NoPartial) {
+  ScanResultCache cache;
+  ASSERT_EQ(ResultVector{}, cache.AddAndGet(ResultVector{}, false));
+  ASSERT_EQ(ResultVector{}, cache.AddAndGet(ResultVector{}, true));
+  int32_t count = 10;
+  ResultVector results{};
+  for (int32_t i = 0; i < count; i++) {
+    results.push_back(CreateResult(CreateCell(i, "cf", "cq1"), false));
+  }
+  ASSERT_EQ(results, cache.AddAndGet(results, false));
+}
+
+TEST(ScanResultCacheTest, Combine1) {
+  ScanResultCache cache;
+  auto prev_result = CreateResult(CreateCell(0, "cf", "cq1"), true);
+  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
+  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
+  auto result3 = CreateResult(CreateCell(1, "cf", "cq3"), true);
+  auto results = cache.AddAndGet(ResultVector{prev_result, result1}, false);
+  ASSERT_EQ(1L, results.size());
+  ASSERT_EQ(prev_result, results[0]);
+
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result3}, false).size());
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{}, true).size());
+
+  results = cache.AddAndGet(ResultVector{}, false);
+  ASSERT_EQ(1, results.size());
+  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(3, results[0]->Cells().size());
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq3")));
+}
+
+TEST(ScanResultCacheTest, Combine2) {
+  ScanResultCache cache;
+  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
+  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
+  auto result3 = CreateResult(CreateCell(1, "cf", "cq3"), true);
+
+  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), true);
+  auto next_to_next_result1 = CreateResult(CreateCell(3, "cf", "cq2"), false);
+
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result3}, false).size());
+
+  auto results = cache.AddAndGet(ResultVector{next_result1}, false);
+  ASSERT_EQ(1, results.size());
+  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(3, results[0]->Cells().size());
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq3")));
+
+  results = cache.AddAndGet(ResultVector{next_to_next_result1}, false);
+  ASSERT_EQ(2, results.size());
+  ASSERT_EQ(2, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(1, results[0]->Cells().size());
+  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(3, folly::to<int32_t>(results[1]->Row()));
+  ASSERT_EQ(1, results[1]->Cells().size());
+  ASSERT_EQ(3, folly::to<int32_t>(*results[1]->Value("cf", "cq2")));
+}
+
+TEST(ScanResultCacheTest, Combine3) {
+  ScanResultCache cache;
+  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
+  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
+  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), false);
+  auto next_to_next_result1 = CreateResult(CreateCell(3, "cf", "cq1"), true);
+
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
+
+  auto results = cache.AddAndGet(ResultVector{next_result1, next_to_next_result1}, false);
+
+  ASSERT_EQ(2, results.size());
+  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(2, results[0]->Cells().size());
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
+  ASSERT_EQ(2, folly::to<int32_t>(results[1]->Row()));
+  ASSERT_EQ(1, results[1]->Cells().size());
+  ASSERT_EQ(2, folly::to<int32_t>(*results[1]->Value("cf", "cq1")));
+
+  results = cache.AddAndGet(ResultVector{}, false);
+
+  ASSERT_EQ(1, results.size());
+  ASSERT_EQ(3, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(1, results[0]->Cells().size());
+  ASSERT_EQ(3, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+}
+
+TEST(ScanResultCacheTest, Combine4) {
+  ScanResultCache cache;
+  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
+  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), false);
+  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), true);
+  auto next_result2 = CreateResult(CreateCell(2, "cf", "cq2"), false);
+
+  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
+
+  auto results = cache.AddAndGet(ResultVector{result2, next_result1}, false);
+
+  ASSERT_EQ(1, results.size());
+  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(2, results[0]->Cells().size());
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
+
+  results = cache.AddAndGet(ResultVector{next_result2}, false);
+
+  ASSERT_EQ(1, results.size());
+  ASSERT_EQ(2, folly::to<int32_t>(results[0]->Row()));
+  ASSERT_EQ(2, results[0]->Cells().size());
+  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
+  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
+}
+
+TEST(ScanResultCacheTest, SizeOf) {
+  std::string e{""};
+  std::string f{"f"};
+  std::string foo{"foo"};
+
+  LOG(INFO) << sizeof(e) << " " << e.capacity();
+  LOG(INFO) << sizeof(f) << " " << f.capacity();
+  LOG(INFO) << sizeof(foo) << " " << foo.capacity();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/scan-result-cache.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/scan-result-cache.cc b/hbase-native-client/src/hbase/client/scan-result-cache.cc
new file mode 100644
index 0000000..e74a7d6
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/scan-result-cache.cc
@@ -0,0 +1,160 @@
+/*
+ * 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 "hbase/client/scan-result-cache.h"
+#include <algorithm>
+#include <iterator>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+/**
+ * Add the given results to cache and get valid results back.
+ * @param results the results of a scan next. Must not be null.
+ * @param is_hearthbeat indicate whether the results is gotten from a heartbeat response.
+ * @return valid results, never null.
+ */
+std::vector<std::shared_ptr<Result>> ScanResultCache::AddAndGet(
+    const std::vector<std::shared_ptr<Result>> &results, bool is_hearthbeat) {
+  // If no results were returned it indicates that either we have the all the partial results
+  // necessary to construct the complete result or the server had to send a heartbeat message
+  // to the client to keep the client-server connection alive
+  if (results.empty()) {
+    // If this response was an empty heartbeat message, then we have not exhausted the region
+    // and thus there may be more partials server side that still need to be added to the partial
+    // list before we form the complete Result
+    if (!partial_results_.empty() && !is_hearthbeat) {
+      return UpdateNumberOfCompleteResultsAndReturn(
+          std::vector<std::shared_ptr<Result>>{Combine()});
+    }
+    return std::vector<std::shared_ptr<Result>>{};
+  }
+  // In every RPC response there should be at most a single partial result. Furthermore, if
+  // there is a partial result, it is guaranteed to be in the last position of the array.
+  auto last = results[results.size() - 1];
+  if (last->Partial()) {
+    if (partial_results_.empty()) {
+      partial_results_.push_back(last);
+      std::vector<std::shared_ptr<Result>> new_results;
+      std::copy_n(results.begin(), results.size() - 1, std::back_inserter(new_results));
+      return UpdateNumberOfCompleteResultsAndReturn(new_results);
+    }
+    // We have only one result and it is partial
+    if (results.size() == 1) {
+      // check if there is a row change
+      if (partial_results_.at(0)->Row() == last->Row()) {
+        partial_results_.push_back(last);
+        return std::vector<std::shared_ptr<Result>>{};
+      }
+      auto complete_result = Combine();
+      partial_results_.push_back(last);
+      return UpdateNumberOfCompleteResultsAndReturn(complete_result);
+    }
+    // We have some complete results
+    auto results_to_return = PrependCombined(results, results.size() - 1);
+    partial_results_.push_back(last);
+    return UpdateNumberOfCompleteResultsAndReturn(results_to_return);
+  }
+  if (!partial_results_.empty()) {
+    return UpdateNumberOfCompleteResultsAndReturn(PrependCombined(results, results.size()));
+  }
+  return UpdateNumberOfCompleteResultsAndReturn(results);
+}
+
+void ScanResultCache::Clear() { partial_results_.clear(); }
+
+std::shared_ptr<Result> ScanResultCache::CreateCompleteResult(
+    const std::vector<std::shared_ptr<Result>> &partial_results) {
+  if (partial_results.empty()) {
+    return std::make_shared<Result>(std::vector<std::shared_ptr<Cell>>{}, false, false, false);
+  }
+  std::vector<std::shared_ptr<Cell>> cells{};
+  bool stale = false;
+  std::string prev_row = "";
+  std::string current_row = "";
+  size_t i = 0;
+  for (const auto &r : partial_results) {
+    current_row = r->Row();
+    if (i != 0 && prev_row != current_row) {
+      throw new std::runtime_error(
+          "Cannot form complete result. Rows of partial results do not match.");
+    }
+    // Ensure that all Results except the last one are marked as partials. The last result
+    // may not be marked as a partial because Results are only marked as partials when
+    // the scan on the server side must be stopped due to reaching the maxResultSize.
+    // Visualizing it makes it easier to understand:
+    // maxResultSize: 2 cells
+    // (-x-) represents cell number x in a row
+    // Example: row1: -1- -2- -3- -4- -5- (5 cells total)
+    // How row1 will be returned by the server as partial Results:
+    // Result1: -1- -2- (2 cells, size limit reached, mark as partial)
+    // Result2: -3- -4- (2 cells, size limit reached, mark as partial)
+    // Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
+    if (i != partial_results.size() - 1 && !r->Partial()) {
+      throw new std::runtime_error("Cannot form complete result. Result is missing partial flag.");
+    }
+    prev_row = current_row;
+    stale = stale || r->Stale();
+    for (const auto &c : r->Cells()) {
+      cells.push_back(c);
+    }
+    i++;
+  }
+
+  return std::make_shared<Result>(cells, false, stale, false);
+}
+
+std::shared_ptr<Result> ScanResultCache::Combine() {
+  auto result = CreateCompleteResult(partial_results_);
+  partial_results_.clear();
+  return result;
+}
+
+std::vector<std::shared_ptr<Result>> ScanResultCache::PrependCombined(
+    const std::vector<std::shared_ptr<Result>> &results, int length) {
+  if (length == 0) {
+    return std::vector<std::shared_ptr<Result>>{Combine()};
+  }
+  // the last part of a partial result may not be marked as partial so here we need to check if
+  // there is a row change.
+  size_t start;
+  if (partial_results_[0]->Row() == results[0]->Row()) {
+    partial_results_.push_back(results[0]);
+    start = 1;
+    length--;
+  } else {
+    start = 0;
+  }
+  std::vector<std::shared_ptr<Result>> prepend_results{};
+  prepend_results.push_back(Combine());
+  std::copy_n(results.begin() + start, length, std::back_inserter(prepend_results));
+  return prepend_results;
+}
+
+std::vector<std::shared_ptr<Result>> ScanResultCache::UpdateNumberOfCompleteResultsAndReturn(
+    const std::shared_ptr<Result> &result) {
+  return UpdateNumberOfCompleteResultsAndReturn(std::vector<std::shared_ptr<Result>>{result});
+}
+
+std::vector<std::shared_ptr<Result>> ScanResultCache::UpdateNumberOfCompleteResultsAndReturn(
+    const std::vector<std::shared_ptr<Result>> &results) {
+  num_complete_rows_ += results.size();
+  return results;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/scan-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/scan-test.cc b/hbase-native-client/src/hbase/client/scan-test.cc
new file mode 100644
index 0000000..ba3a029
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/scan-test.cc
@@ -0,0 +1,228 @@
+/*
+ * 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 <gtest/gtest.h>
+#include <limits>
+
+#include "hbase/client/scan.h"
+
+using hbase::Get;
+using hbase::Scan;
+
+void CheckFamilies(Scan &scan) {
+  EXPECT_EQ(false, scan.HasFamilies());
+  scan.AddFamily("family-1");
+  EXPECT_EQ(true, scan.HasFamilies());
+  EXPECT_EQ(1, scan.FamilyMap().size());
+  for (const auto &family : scan.FamilyMap()) {
+    EXPECT_STREQ("family-1", family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+  }
+  // Not allowed to add the same CF.
+  scan.AddFamily("family-1");
+  EXPECT_EQ(1, scan.FamilyMap().size());
+  scan.AddFamily("family-2");
+  EXPECT_EQ(2, scan.FamilyMap().size());
+  scan.AddFamily("family-3");
+  EXPECT_EQ(3, scan.FamilyMap().size());
+  int i = 1;
+  for (const auto &family : scan.FamilyMap()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+    i += 1;
+  }
+
+  scan.AddColumn("family-1", "column-1");
+  scan.AddColumn("family-1", "column-2");
+  scan.AddColumn("family-1", "");
+  scan.AddColumn("family-1", "column-3");
+  scan.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, scan.FamilyMap().size());
+  auto it = scan.FamilyMap().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, scan.FamilyMap().end());
+}
+
+void CheckFamiliesAfterCopy(const Scan &scan) {
+  EXPECT_EQ(true, scan.HasFamilies());
+  EXPECT_EQ(3, scan.FamilyMap().size());
+  int i = 1;
+  for (const auto &family : scan.FamilyMap()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    i += 1;
+  }
+  // Check if the alreaday added CF's and CQ's are as expected
+  auto it = scan.FamilyMap().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, scan.FamilyMap().end());
+}
+
+void ScanMethods(Scan &scan) {
+  scan.SetReversed(true);
+  EXPECT_EQ(true, scan.IsReversed());
+  scan.SetReversed(false);
+  EXPECT_EQ(false, scan.IsReversed());
+
+  std::string start_row("start-row");
+  std::string stop_row("stop-row");
+  scan.SetStartRow(start_row);
+  EXPECT_EQ(start_row, scan.StartRow());
+
+  scan.SetStopRow(stop_row);
+  EXPECT_EQ(stop_row, scan.StopRow());
+
+  scan.SetCaching(3);
+  EXPECT_EQ(3, scan.Caching());
+
+  scan.SetConsistency(hbase::pb::Consistency::STRONG);
+  EXPECT_EQ(hbase::pb::Consistency::STRONG, scan.Consistency());
+  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, scan.Consistency());
+
+  scan.SetCacheBlocks(true);
+  EXPECT_EQ(true, scan.CacheBlocks());
+  scan.SetCacheBlocks(false);
+  EXPECT_EQ(false, scan.CacheBlocks());
+
+  scan.SetAllowPartialResults(true);
+  EXPECT_EQ(true, scan.AllowPartialResults());
+  scan.SetAllowPartialResults(false);
+  EXPECT_EQ(false, scan.AllowPartialResults());
+
+  scan.SetLoadColumnFamiliesOnDemand(true);
+  EXPECT_EQ(true, scan.LoadColumnFamiliesOnDemand());
+  scan.SetLoadColumnFamiliesOnDemand(false);
+  EXPECT_EQ(false, scan.LoadColumnFamiliesOnDemand());
+
+  scan.SetMaxVersions();
+  EXPECT_EQ(1, scan.MaxVersions());
+  scan.SetMaxVersions(20);
+  EXPECT_EQ(20, scan.MaxVersions());
+
+  scan.SetMaxResultSize(1024);
+  EXPECT_EQ(1024, scan.MaxResultSize());
+
+  // Test initial values
+  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(std::numeric_limits<int64_t>::max(), scan.Timerange().MaxTimeStamp());
+
+  // Set & Test new values using TimeRange and TimeStamp
+  scan.SetTimeRange(1000, 2000);
+  EXPECT_EQ(1000, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(2000, scan.Timerange().MaxTimeStamp());
+  scan.SetTimeStamp(0);
+  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(1, scan.Timerange().MaxTimeStamp());
+
+  // Test some exceptions
+  ASSERT_THROW(scan.SetTimeRange(-1000, 2000), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeRange(1000, -2000), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeRange(1000, 200), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeStamp(std::numeric_limits<int64_t>::max()), std::runtime_error);
+}
+
+TEST(Scan, Object) {
+  Scan scan;
+  ScanMethods(scan);
+  CheckFamilies(scan);
+
+  // Resetting TimeRange values so that the copy construction and assignment
+  // operator tests pass.
+  scan.SetTimeRange(0, std::numeric_limits<int64_t>::max());
+  Scan scancp(scan);
+  ScanMethods(scancp);
+  CheckFamiliesAfterCopy(scancp);
+
+  Scan scaneq;
+  scaneq = scan;
+  ScanMethods(scaneq);
+  CheckFamiliesAfterCopy(scaneq);
+}
+
+TEST(Scan, WithStartRow) {
+  Scan("row-test");
+  Scan scan("row-test");
+  ScanMethods(scan);
+  CheckFamilies(scan);
+}
+
+TEST(Scan, WithStartAndStopRow) {
+  Scan("start-row", "stop-row");
+  Scan scan("start-row", "stop-row");
+  ScanMethods(scan);
+  CheckFamilies(scan);
+}
+
+TEST(Scan, FromGet) {
+  std::string row_str = "row-test";
+  Get get = Get(row_str);
+
+  get.SetCacheBlocks(true);
+  get.SetMaxVersions(5);
+  get.AddFamily("family-1");
+  get.AddFamily("family-1");
+  get.AddFamily("family-2");
+  get.AddFamily("family-3");
+  get.AddColumn("family-1", "column-1");
+  get.AddColumn("family-1", "column-2");
+  get.AddColumn("family-1", "");
+  get.AddColumn("family-1", "column-3");
+  get.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, get.FamilyMap().size());
+
+  Scan scan(get);
+  ScanMethods(scan);
+  CheckFamiliesAfterCopy(scan);
+}
+
+TEST(Scan, Exception) {
+  std::string row(std::numeric_limits<int16_t>::max() + 1, 'X');
+  ASSERT_THROW(Scan tmp(row), std::runtime_error);
+  ASSERT_THROW(Scan tmp(""), std::runtime_error);
+}


[10/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/security/user.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/security/user.h b/hbase-native-client/security/user.h
deleted file mode 100644
index 307fc61..0000000
--- a/hbase-native-client/security/user.h
+++ /dev/null
@@ -1,46 +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.
- *
- */
-#pragma once
-
-#include <glog/logging.h>
-#include <mutex>
-#include <string>
-#include "core/configuration.h"
-
-namespace hbase {
-namespace security {
-static constexpr const char* kKerberos = "kerberos";
-class User {
- public:
-  explicit User(const std::string& user_name) : user_name_(user_name) {}
-  virtual ~User() = default;
-
-  std::string user_name() { return user_name_; }
-
-  static std::shared_ptr<User> defaultUser() { return std::make_shared<User>("__drwho"); }
-
-  static bool IsSecurityEnabled(const Configuration& conf) {
-    return conf.Get("hbase.security.authentication", "").compare(kKerberos) == 0;
-  }
-
- private:
-  std::string user_name_;
-};
-}
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/BUCK b/hbase-native-client/serde/BUCK
deleted file mode 100644
index a765884..0000000
--- a/hbase-native-client/serde/BUCK
+++ /dev/null
@@ -1,96 +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.
-
-cxx_library(
-    name="serde",
-    exported_headers=[
-        "cell-scanner.h",
-        "cell-outputstream.h",
-        "codec.h",
-        "region-info.h",
-        "rpc-serde.h",
-        "server-name.h",
-        "table-name.h",
-        "zk.h",
-    ],
-    srcs=[
-        "rpc-serde.cc",
-        "zk.cc",
-    ],
-    deps=[
-        "//if:if", "//third-party:folly", "//utils:utils", "//security:security"
-    ],
-    tests=[
-        ":client-deserializer-test",
-        ":client-serializer-test",
-        ":server-name-test",
-        ":table-name-test",
-        ":zk-deserializer-test",
-        ":region-info-deserializer-test",
-    ],
-    compiler_flags=['-Weffc++'],
-    visibility=[
-        'PUBLIC',
-    ],)
-cxx_test(
-    name="table-name-test",
-    srcs=[
-        "table-name-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)
-cxx_test(
-    name="server-name-test",
-    srcs=[
-        "server-name-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)
-cxx_test(
-    name="client-serializer-test",
-    srcs=[
-        "client-serializer-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)
-cxx_test(
-    name="client-deserializer-test",
-    srcs=[
-        "client-deserializer-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)
-cxx_test(
-    name="zk-deserializer-test",
-    srcs=[
-        "zk-deserializer-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)
-cxx_test(
-    name="region-info-deserializer-test",
-    srcs=[
-        "region-info-deserializer-test.cc",
-    ],
-    deps=[
-        ":serde",
-    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/cell-outputstream.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/cell-outputstream.h b/hbase-native-client/serde/cell-outputstream.h
deleted file mode 100644
index 963dd31..0000000
--- a/hbase-native-client/serde/cell-outputstream.h
+++ /dev/null
@@ -1,53 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-
-namespace hbase {
-
-class Cell;
-
-/**
- * @brief Encoder / Decoder for Cells.
- */
-class CellOutputStream {
- public:
-  virtual ~CellOutputStream() {}
-
-  /**
-   * Implementation must copy the entire state of the Cell. If the written Cell is modified
-   * immediately after the write method returns, the modifications must have absolutely no effect
-   * on the copy of the Cell that was added in the write.
-   * @param cell Cell to write out
-   * @throws IOException
-   */
-  virtual void Write(const Cell& cell) = 0;
-
-  /**
-   * Let the implementation decide what to do.  Usually means writing accumulated data into a
-   * byte[] that can then be read from the implementation to be sent to disk, put in the block
-   * cache, or sent over the network.
-   * @throws IOException
-   */
-  virtual void Flush() = 0;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/cell-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/cell-scanner.h b/hbase-native-client/serde/cell-scanner.h
deleted file mode 100644
index fe4a249..0000000
--- a/hbase-native-client/serde/cell-scanner.h
+++ /dev/null
@@ -1,50 +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.
- *
- */
-
-#pragma once
-#include <folly/io/IOBuf.h>
-#include <memory>
-
-namespace hbase {
-
-class Cell;
-
-/**
- * @brief Interface for iterating over a sequence of Cells
- */
-class CellScanner {
- public:
-  virtual ~CellScanner() {}
-
-  /**
-   * @brief This method will be used to iterate the cells.
-   * Typical usage will be :-
-   * while(cell_scanner.Advance()){
-   *  auto current_cell = cell_scanner.Current();
-   * }
-   */
-  virtual bool Advance() = 0;
-
-  /**
-   * @brief returns the current cell
-   */
-  virtual const std::shared_ptr<Cell> Current() const = 0;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/client-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/client-deserializer-test.cc b/hbase-native-client/serde/client-deserializer-test.cc
deleted file mode 100644
index 1856047..0000000
--- a/hbase-native-client/serde/client-deserializer-test.cc
+++ /dev/null
@@ -1,64 +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 <folly/io/IOBuf.h>
-#include <gtest/gtest.h>
-
-#include "if/Client.pb.h"
-#include "rpc-serde.h"
-
-using namespace hbase;
-using folly::IOBuf;
-using hbase::pb::GetRequest;
-using hbase::pb::RegionSpecifier;
-using hbase::pb::RegionSpecifier_RegionSpecifierType;
-
-TEST(TestRpcSerde, TestReturnFalseOnNullPtr) {
-  RpcSerde deser{nullptr};
-  ASSERT_LT(deser.ParseDelimited(nullptr, nullptr), 0);
-}
-
-TEST(TestRpcSerde, TestReturnFalseOnBadInput) {
-  RpcSerde deser{nullptr};
-  auto buf = IOBuf::copyBuffer("test");
-  GetRequest gr;
-
-  ASSERT_LT(deser.ParseDelimited(buf.get(), &gr), 0);
-}
-
-TEST(TestRpcSerde, TestGoodGetRequestFullRoundTrip) {
-  GetRequest in;
-  RpcSerde ser{nullptr};
-  RpcSerde deser{nullptr};
-
-  // fill up the GetRequest.
-  in.mutable_region()->set_value("test_region_id");
-  in.mutable_region()->set_type(
-      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_ENCODED_REGION_NAME);
-  in.mutable_get()->set_row("test_row");
-
-  // Create the buffer
-  auto buf = ser.SerializeDelimited(in);
-
-  GetRequest out;
-
-  int used_bytes = deser.ParseDelimited(buf.get(), &out);
-
-  ASSERT_GT(used_bytes, 0);
-  ASSERT_EQ(used_bytes, buf->length());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/client-serializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/client-serializer-test.cc b/hbase-native-client/serde/client-serializer-test.cc
deleted file mode 100644
index 306f2c2..0000000
--- a/hbase-native-client/serde/client-serializer-test.cc
+++ /dev/null
@@ -1,75 +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 <gtest/gtest.h>
-
-#include <folly/io/Cursor.h>
-
-#include <string>
-
-#include "if/HBase.pb.h"
-#include "if/RPC.pb.h"
-#include "rpc-serde.h"
-
-using namespace hbase;
-using namespace hbase::pb;
-using namespace folly;
-using namespace folly::io;
-
-TEST(RpcSerdeTest, PreambleIncludesHBas) {
-  RpcSerde ser{nullptr};
-  auto buf = ser.Preamble(false);
-  const char *p = reinterpret_cast<const char *>(buf->data());
-  // Take the first for chars and make sure they are the
-  // magic string
-  EXPECT_EQ("HBas", std::string(p, 4));
-
-  EXPECT_EQ(6, buf->computeChainDataLength());
-}
-
-TEST(RpcSerdeTest, PreambleIncludesVersion) {
-  RpcSerde ser{nullptr};
-  auto buf = ser.Preamble(false);
-  EXPECT_EQ(0, static_cast<const uint8_t *>(buf->data())[4]);
-  EXPECT_EQ(80, static_cast<const uint8_t *>(buf->data())[5]);
-}
-
-TEST(RpcSerdeTest, TestHeaderLengthPrefixed) {
-  RpcSerde ser{nullptr};
-  auto header = ser.Header("elliott");
-
-  // The header should be prefixed by 4 bytes of length.
-  EXPECT_EQ(4, header->length());
-  EXPECT_TRUE(header->length() < header->computeChainDataLength());
-  EXPECT_TRUE(header->isChained());
-
-  // Now make sure the length is correct.
-  Cursor cursor(header.get());
-  auto prefixed_len = cursor.readBE<uint32_t>();
-  EXPECT_EQ(prefixed_len, header->next()->length());
-}
-
-TEST(RpcSerdeTest, TestHeaderDecode) {
-  RpcSerde ser{nullptr};
-  auto buf = ser.Header("elliott");
-  auto header_buf = buf->next();
-  ConnectionHeader h;
-
-  EXPECT_TRUE(h.ParseFromArray(header_buf->data(), header_buf->length()));
-  EXPECT_EQ("elliott", h.user_info().effective_user());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/codec.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/codec.h b/hbase-native-client/serde/codec.h
deleted file mode 100644
index 64807dc..0000000
--- a/hbase-native-client/serde/codec.h
+++ /dev/null
@@ -1,50 +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.
- *
- */
-
-#pragma once
-
-#include <folly/io/IOBuf.h>
-#include <memory>
-
-#include "serde/cell-outputstream.h"
-#include "serde/cell-scanner.h"
-
-namespace hbase {
-
-/**
- * @brief Encoder / Decoder for Cells.
- */
-class Codec {
- public:
-  virtual ~Codec() {}
-
-  class Encoder : public CellOutputStream {};
-
-  class Decoder : public CellScanner {};
-
-  virtual std::unique_ptr<Encoder> CreateEncoder() = 0;
-  virtual std::unique_ptr<Decoder> CreateDecoder(std::unique_ptr<folly::IOBuf> cell_block,
-                                                 uint32_t cell_block_start_offset,
-                                                 uint32_t cell_block_length) = 0;
-
-  /** @brief returns the java class name corresponding to this Codec implementation */
-  virtual const char* java_class_name() const = 0;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/region-info-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/region-info-deserializer-test.cc b/hbase-native-client/serde/region-info-deserializer-test.cc
deleted file mode 100644
index 5cb8482..0000000
--- a/hbase-native-client/serde/region-info-deserializer-test.cc
+++ /dev/null
@@ -1,53 +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 "serde/region-info.h"
-
-#include <gtest/gtest.h>
-
-#include <string>
-
-#include "if/HBase.pb.h"
-#include "serde/table-name.h"
-
-using std::string;
-using hbase::pb::RegionInfo;
-using hbase::pb::TableName;
-
-TEST(TestRegionInfoDesializer, TestDeserialize) {
-  string ns{"test_ns"};
-  string tn{"table_name"};
-  string start_row{"AAAAAA"};
-  string stop_row{"BBBBBBBBBBBB"};
-  uint64_t region_id = 2345678;
-
-  RegionInfo ri_out;
-  ri_out.set_region_id(region_id);
-  ri_out.mutable_table_name()->set_namespace_(ns);
-  ri_out.mutable_table_name()->set_qualifier(tn);
-  ri_out.set_start_key(start_row);
-  ri_out.set_end_key(stop_row);
-
-  string header{"PBUF"};
-  string ser = header + ri_out.SerializeAsString();
-
-  auto out = folly::to<RegionInfo>(ser);
-
-  EXPECT_EQ(region_id, out.region_id());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/region-info.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/region-info.h b/hbase-native-client/serde/region-info.h
deleted file mode 100644
index 8010042..0000000
--- a/hbase-native-client/serde/region-info.h
+++ /dev/null
@@ -1,44 +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.
- *
- */
-
-#pragma once
-
-#include <folly/Conv.h>
-#include <boost/algorithm/string/predicate.hpp>
-
-#include <string>
-
-#include "if/HBase.pb.h"
-
-namespace hbase {
-namespace pb {
-template <class String>
-void parseTo(String in, RegionInfo &out) {
-  // TODO(eclark): there has to be something better.
-  std::string s = folly::to<std::string>(in);
-
-  if (!boost::starts_with(s, "PBUF")) {
-    throw std::runtime_error("Region Info field doesn't contain preamble");
-  }
-  if (!out.ParseFromArray(s.data() + 4, s.size() - 4)) {
-    throw std::runtime_error("Bad protobuf for RegionInfo");
-  }
-}
-}  // namespace pb
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/rpc-serde.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/rpc-serde.cc b/hbase-native-client/serde/rpc-serde.cc
deleted file mode 100644
index 70a57e8..0000000
--- a/hbase-native-client/serde/rpc-serde.cc
+++ /dev/null
@@ -1,261 +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 <folly/Conv.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>
-#include <google/protobuf/message.h>
-#include <boost/algorithm/string.hpp>
-
-#include <utility>
-
-#include "if/RPC.pb.h"
-#include "rpc-serde.h"
-#include "utils/version.h"
-
-using folly::IOBuf;
-using folly::io::RWPrivateCursor;
-using google::protobuf::Message;
-using google::protobuf::io::ArrayInputStream;
-using google::protobuf::io::ArrayOutputStream;
-using google::protobuf::io::CodedInputStream;
-using google::protobuf::io::CodedOutputStream;
-using google::protobuf::io::ZeroCopyOutputStream;
-
-using namespace hbase::pb;
-
-namespace hbase {
-
-static const std::string PREAMBLE = "HBas";
-static const std::string INTERFACE = "ClientService";
-static const uint8_t RPC_VERSION = 0;
-static const uint8_t DEFAULT_AUTH_TYPE = 80;
-static const uint8_t KERBEROS_AUTH_TYPE = 81;
-
-int RpcSerde::ParseDelimited(const IOBuf *buf, Message *msg) {
-  if (buf == nullptr || msg == nullptr) {
-    return -2;
-  }
-
-  DCHECK(!buf->isChained());
-
-  ArrayInputStream ais{buf->data(), static_cast<int>(buf->length())};
-  CodedInputStream coded_stream{&ais};
-
-  uint32_t msg_size;
-
-  // Try and read the varint.
-  if (coded_stream.ReadVarint32(&msg_size) == false) {
-    FB_LOG_EVERY_MS(ERROR, 1000) << "Unable to read a var uint32_t";
-    return -3;
-  }
-
-  coded_stream.PushLimit(msg_size);
-  // Parse the message.
-  if (msg->MergeFromCodedStream(&coded_stream) == false) {
-    FB_LOG_EVERY_MS(ERROR, 1000) << "Unable to read a protobuf message from data.";
-    return -4;
-  }
-
-  // Make sure all the data was consumed.
-  if (coded_stream.ConsumedEntireMessage() == false) {
-    FB_LOG_EVERY_MS(ERROR, 1000) << "Orphaned data left after reading protobuf message";
-    return -5;
-  }
-
-  return coded_stream.CurrentPosition();
-}
-
-RpcSerde::RpcSerde() {}
-
-RpcSerde::RpcSerde(std::shared_ptr<Codec> codec) : codec_(codec) {}
-
-std::unique_ptr<IOBuf> RpcSerde::Preamble(bool secure) {
-  auto magic = IOBuf::copyBuffer(PREAMBLE, 0, 2);
-  magic->append(2);
-  RWPrivateCursor c(magic.get());
-  c.skip(4);
-  // Version
-  c.write(RPC_VERSION);
-  if (secure) {
-    // for now support only KERBEROS (DIGEST is not supported)
-    c.write(KERBEROS_AUTH_TYPE);
-  } else {
-    c.write(DEFAULT_AUTH_TYPE);
-  }
-  return magic;
-}
-
-std::unique_ptr<IOBuf> RpcSerde::Header(const std::string &user) {
-  pb::ConnectionHeader h;
-
-  // TODO(eclark): Make this not a total lie.
-  h.mutable_user_info()->set_effective_user(user);
-  // The service name that we want to talk to.
-  //
-  // Right now we're completely ignoring the service interface.
-  // That may or may not be the correct thing to do.
-  // It worked for a while with the java client; until it
-  // didn't.
-  // TODO: send the service name and user from the RpcClient
-  h.set_service_name(INTERFACE);
-
-  std::unique_ptr<pb::VersionInfo> version_info = CreateVersionInfo();
-
-  h.set_allocated_version_info(version_info.release());
-
-  if (codec_ != nullptr) {
-    h.set_cell_block_codec_class(codec_->java_class_name());
-  }
-  return PrependLength(SerializeMessage(h));
-}
-
-std::unique_ptr<pb::VersionInfo> RpcSerde::CreateVersionInfo() {
-  std::unique_ptr<pb::VersionInfo> version_info = std::make_unique<pb::VersionInfo>();
-  version_info->set_user(Version::user);
-  version_info->set_revision(Version::revision);
-  version_info->set_url(Version::url);
-  version_info->set_date(Version::date);
-  version_info->set_src_checksum(Version::src_checksum);
-  version_info->set_version(Version::version);
-
-  std::string version{Version::version};
-  std::vector<std::string> version_parts;
-  boost::split(version_parts, version, boost::is_any_of("."), boost::token_compress_on);
-  uint32_t major_version = 0, minor_version = 0;
-  if (version_parts.size() >= 2) {
-    version_info->set_version_major(folly::to<uint32_t>(version_parts[0]));
-    version_info->set_version_minor(folly::to<uint32_t>(version_parts[1]));
-  }
-
-  VLOG(1) << "Client VersionInfo:" << version_info->ShortDebugString();
-  return version_info;
-}
-
-std::unique_ptr<IOBuf> RpcSerde::Request(const uint32_t call_id, const std::string &method,
-                                         const Message *msg) {
-  pb::RequestHeader rq;
-  rq.set_method_name(method);
-  rq.set_call_id(call_id);
-  rq.set_request_param(msg != nullptr);
-  auto ser_header = SerializeDelimited(rq);
-  if (msg != nullptr) {
-    auto ser_req = SerializeDelimited(*msg);
-    ser_header->appendChain(std::move(ser_req));
-  }
-
-  return PrependLength(std::move(ser_header));
-}
-
-std::unique_ptr<folly::IOBuf> RpcSerde::Response(const uint32_t call_id,
-                                                 const google::protobuf::Message *msg) {
-  pb::ResponseHeader rh;
-  rh.set_call_id(call_id);
-  auto ser_header = SerializeDelimited(rh);
-  auto ser_resp = SerializeDelimited(*msg);
-  ser_header->appendChain(std::move(ser_resp));
-
-  return PrependLength(std::move(ser_header));
-}
-
-std::unique_ptr<folly::IOBuf> RpcSerde::Response(const uint32_t call_id,
-                                                 const google::protobuf::Message *msg,
-                                                 const folly::exception_wrapper &exception) {
-  /* create ResponseHeader */
-  pb::ResponseHeader rh;
-  rh.set_call_id(call_id);
-
-  /* create ExceptionResponse */
-  if (bool(exception)) {
-    VLOG(1) << "packing ExceptionResponse";
-    auto exception_response = new pb::ExceptionResponse();
-    exception_response->set_exception_class_name(exception.class_name().c_str());
-    exception_response->set_stack_trace(exception.what().c_str());
-    rh.set_allocated_exception(exception_response);
-  }
-
-  /* serialize Response header and body */
-  auto ser_header = SerializeDelimited(rh);
-  auto ser_resp = SerializeDelimited(*msg);
-  ser_header->appendChain(std::move(ser_resp));
-
-  VLOG(3) << "Converted hbase::Response to folly::IOBuf";
-  return PrependLength(std::move(ser_header));
-}
-
-std::unique_ptr<CellScanner> RpcSerde::CreateCellScanner(std::unique_ptr<folly::IOBuf> buf,
-                                                         uint32_t offset, uint32_t length) {
-  if (codec_ == nullptr) {
-    return nullptr;
-  }
-  return codec_->CreateDecoder(std::move(buf), offset, length);
-}
-
-std::unique_ptr<IOBuf> RpcSerde::PrependLength(std::unique_ptr<IOBuf> msg) {
-  // Java ints are 4 long. So create a buffer that large
-  auto len_buf = IOBuf::create(4);
-  // Then make those bytes visible.
-  len_buf->append(4);
-
-  RWPrivateCursor c(len_buf.get());
-  // Get the size of the data to be pushed out the network.
-  auto size = msg->computeChainDataLength();
-
-  // Write the length to this IOBuf.
-  c.writeBE(static_cast<uint32_t>(size));
-
-  // Then attach the origional to the back of len_buf
-  len_buf->appendChain(std::move(msg));
-  return len_buf;
-}
-
-std::unique_ptr<IOBuf> RpcSerde::SerializeDelimited(const Message &msg) {
-  // Get the buffer size needed for just the message.
-  int msg_size = msg.ByteSize();
-  int buf_size = CodedOutputStream::VarintSize32(msg_size) + msg_size;
-
-  // Create a buffer big enough to hold the varint and the object.
-  auto buf = IOBuf::create(buf_size);
-  buf->append(buf_size);
-
-  // Create the array output stream.
-  ArrayOutputStream aos{buf->writableData(), static_cast<int>(buf->length())};
-  // Wrap the ArrayOuputStream in the coded output stream to allow writing
-  // Varint32
-  CodedOutputStream cos{&aos};
-
-  // Write out the size.
-  cos.WriteVarint32(msg_size);
-
-  // Now write the rest out.
-  // We're using the protobuf output streams here to keep track
-  // of where in the output array we are rather than IOBuf.
-  msg.SerializeWithCachedSizesToArray(cos.GetDirectBufferForNBytesAndAdvance(msg_size));
-
-  // Return the buffer.
-  return buf;
-}
-// TODO(eclark): Make this 1 copy.
-std::unique_ptr<IOBuf> RpcSerde::SerializeMessage(const Message &msg) {
-  auto buf = IOBuf::copyBuffer(msg.SerializeAsString());
-  return buf;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/rpc-serde.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/rpc-serde.h b/hbase-native-client/serde/rpc-serde.h
deleted file mode 100644
index 6941f62..0000000
--- a/hbase-native-client/serde/rpc-serde.h
+++ /dev/null
@@ -1,155 +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.
- *
- */
-#pragma once
-
-#include <memory>
-#include <string>
-
-#include <folly/ExceptionWrapper.h>
-#include "if/HBase.pb.h"
-#include "serde/cell-scanner.h"
-#include "serde/codec.h"
-
-using namespace folly;
-// Forward
-namespace folly {
-class IOBuf;
-}
-namespace google {
-namespace protobuf {
-class Message;
-}
-}
-
-namespace hbase {
-
-/**
- * @brief Class for serializing a deserializing rpc formatted data.
- *
- * RpcSerde is the one stop shop for reading/writing data to HBase daemons.
- * It should throw exceptions if anything goes wrong.
- */
-class RpcSerde {
- public:
-  RpcSerde();
-  /**
-   * Constructor assumes the default auth type.
-   */
-  RpcSerde(std::shared_ptr<Codec> codec);
-
-  /**
-   * Destructor. This is provided just for testing purposes.
-   */
-  virtual ~RpcSerde() = default;
-
-  /**
-   * Pase a message in the delimited format.
-   *
-   * A message in delimited format consists of the following:
-   *
-   * - a protobuf var int32.
-   * - A protobuf object serialized.
-   */
-  int ParseDelimited(const folly::IOBuf *buf, google::protobuf::Message *msg);
-
-  /**
-   * Create a new connection preamble in a new IOBuf.
-   */
-  static std::unique_ptr<folly::IOBuf> Preamble(bool secure);
-
-  /**
-   * Create the header protobuf object and serialize it to a new IOBuf.
-   * Header is in the following format:
-   *
-   * - Big endian length
-   * - ConnectionHeader object serialized out.
-   */
-  std::unique_ptr<folly::IOBuf> Header(const std::string &user);
-
-  /**
-   * Take ownership of the passed buffer, and create a CellScanner using the
-   * Codec class to parse Cells out of the wire.
-   */
-  std::unique_ptr<CellScanner> CreateCellScanner(std::unique_ptr<folly::IOBuf> buf, uint32_t offset,
-                                                 uint32_t length);
-
-  /**
-   * Serialize a request message into a protobuf.
-   * Request consists of:
-   *
-   * - Big endian length
-   * - RequestHeader object
-   * - The passed in Message object
-   */
-  std::unique_ptr<folly::IOBuf> Request(const uint32_t call_id, const std::string &method,
-                                        const google::protobuf::Message *msg);
-
-  /**
-     * Serialize a response message into a protobuf.
-     * Request consists of:
-     *
-     * - Big endian length
-     * - ResponseHeader object
-     * - The passed in Message object
-     */
-  std::unique_ptr<folly::IOBuf> Response(const uint32_t call_id,
-                                         const google::protobuf::Message *msg);
-
-  /**
-   * Serialize a response message into a protobuf.
-   * Request consists of:
-   *
-   * - Big endian length
-   * - ResponseHeader object
-   * - The passed in hbase::Response object
-   */
-  std::unique_ptr<folly::IOBuf> Response(const uint32_t call_id,
-                                         const google::protobuf::Message *msg,
-                                         const folly::exception_wrapper &exception);
-
-  /**
-   * Serialize a message in the delimited format.
-   * Delimited format consists of the following:
-   *
-   * - A protobuf var int32
-   * - The message object seriailized after that.
-   */
-  std::unique_ptr<folly::IOBuf> SerializeDelimited(const google::protobuf::Message &msg);
-
-  /**
-   * Serilalize a message. This does not add any length prepend.
-   */
-  std::unique_ptr<folly::IOBuf> SerializeMessage(const google::protobuf::Message &msg);
-
-  /**
-   * Prepend a length IOBuf to the given IOBuf chain.
-   * This involves no copies or moves of the passed in data.
-   */
-  std::unique_ptr<folly::IOBuf> PrependLength(std::unique_ptr<folly::IOBuf> msg);
-
- public:
-  static constexpr const char *HBASE_CLIENT_RPC_TEST_MODE = "hbase.client.rpc.test.mode";
-  static constexpr const bool DEFAULT_HBASE_CLIENT_RPC_TEST_MODE = false;
-
- private:
-  /* data */
-  std::shared_ptr<Codec> codec_;
-  std::unique_ptr<pb::VersionInfo> CreateVersionInfo();
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/server-name-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/server-name-test.cc b/hbase-native-client/serde/server-name-test.cc
deleted file mode 100644
index 87c493a..0000000
--- a/hbase-native-client/serde/server-name-test.cc
+++ /dev/null
@@ -1,47 +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 "serde/server-name.h"
-
-#include <gtest/gtest.h>
-#include <string>
-
-using hbase::pb::ServerName;
-
-TEST(TestServerName, TestMakeServerName) {
-  auto sn = folly::to<ServerName>("test:123");
-
-  ASSERT_EQ("test", sn.host_name());
-  ASSERT_EQ(123, sn.port());
-}
-
-TEST(TestServerName, TestIps) {
-  auto sn = folly::to<ServerName>("127.0.0.1:999");
-  ASSERT_EQ("127.0.0.1", sn.host_name());
-  ASSERT_EQ(999, sn.port());
-}
-
-TEST(TestServerName, TestThrow) { ASSERT_ANY_THROW(folly::to<ServerName>("Ther's no colon here")); }
-
-TEST(TestServerName, TestIPV6) {
-  auto sn = folly::to<ServerName>("[::::1]:123");
-
-  ASSERT_EQ("[::::1]", sn.host_name());
-  ASSERT_EQ(123, sn.port());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/server-name.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/server-name.h b/hbase-native-client/serde/server-name.h
deleted file mode 100644
index 41e3c77..0000000
--- a/hbase-native-client/serde/server-name.h
+++ /dev/null
@@ -1,47 +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.
- *
- */
-
-#pragma once
-
-#include <folly/Conv.h>
-#include <folly/String.h>
-
-#include <string>
-
-#include "if/HBase.pb.h"
-
-namespace hbase {
-namespace pb {
-
-template <class String>
-void parseTo(String in, ServerName &out) {
-  // TODO see about getting rsplit into folly.
-  std::string s = folly::to<std::string>(in);
-
-  auto delim = s.rfind(":");
-  if (delim == std::string::npos) {
-    throw std::runtime_error("Couldn't parse server name");
-  }
-  out.set_host_name(s.substr(0, delim));
-  // Now keep everything after the : (delim + 1) to the end.
-  out.set_port(folly::to<int>(s.substr(delim + 1)));
-}
-
-}  // namespace pb
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/table-name-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/table-name-test.cc b/hbase-native-client/serde/table-name-test.cc
deleted file mode 100644
index 877d522..0000000
--- a/hbase-native-client/serde/table-name-test.cc
+++ /dev/null
@@ -1,54 +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 <folly/Conv.h>
-#include <gtest/gtest.h>
-
-#include <string>
-
-#include "serde/table-name.h"
-
-using namespace hbase;
-using hbase::pb::TableName;
-
-TEST(TestTableName, TestToStringNoDefault) {
-  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;
-  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);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/table-name.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/table-name.h b/hbase-native-client/serde/table-name.h
deleted file mode 100644
index 3594802..0000000
--- a/hbase-native-client/serde/table-name.h
+++ /dev/null
@@ -1,58 +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.
- *
- */
-#pragma once
-
-#include <folly/Conv.h>
-#include <folly/String.h>
-
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "if/HBase.pb.h"
-
-namespace hbase {
-namespace pb {
-
-// Provide folly::to<std::string>(TableName);
-template <class String>
-void toAppend(const TableName &in, String *result) {
-  if (!in.has_namespace_() || in.namespace_() == "default") {
-    folly::toAppend(in.qualifier(), result);
-  } else {
-    folly::toAppend(in.namespace_(), ':', in.qualifier(), result);
-  }
-}
-
-template <class String>
-void parseTo(String in, TableName &out) {
-  std::vector<std::string> v;
-  folly::split(":", in, v);
-
-  if (v.size() == 1) {
-    out.set_namespace_("default");
-    out.set_qualifier(v[0]);
-  } else {
-    out.set_namespace_(v[0]);
-    out.set_qualifier(v[1]);
-  }
-}
-
-}  // namespace pb
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/zk-deserializer-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/zk-deserializer-test.cc b/hbase-native-client/serde/zk-deserializer-test.cc
deleted file mode 100644
index f07eecf..0000000
--- a/hbase-native-client/serde/zk-deserializer-test.cc
+++ /dev/null
@@ -1,123 +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 "serde/zk.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());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/zk.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/zk.cc b/hbase-native-client/serde/zk.cc
deleted file mode 100644
index a71eb87..0000000
--- a/hbase-native-client/serde/zk.cc
+++ /dev/null
@@ -1,77 +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 "serde/zk.h"
-
-#include <folly/io/Cursor.h>
-#include <folly/io/IOBuf.h>
-#include <google/protobuf/message.h>
-
-#include <string>
-
-using std::runtime_error;
-
-namespace hbase {
-
-static const std::string MAGIC_STRING = "PBUF";
-
-bool ZkDeserializer::Parse(folly::IOBuf *buf, google::protobuf::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.
-
-  folly::io::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;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/serde/zk.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/serde/zk.h b/hbase-native-client/serde/zk.h
deleted file mode 100644
index 5cadec2..0000000
--- a/hbase-native-client/serde/zk.h
+++ /dev/null
@@ -1,49 +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.
- *
- */
-#pragma once
-
-namespace google {
-namespace protobuf {
-class Message;
-}
-}
-namespace folly {
-class IOBuf;
-}
-
-namespace hbase {
-
-/** @brief A class to convert data from ZooKeeper to other formats.
- *
- * This class will convert data to and from Zookeeper into protobuf objects.
- *
- */
-class ZkDeserializer {
- public:
-  /**
-   * Merge the data from a buffer into a given message.
-   *
-   * @param buf Naked pointer to iobuf containing data read from zookeeper.
-   * @param out Naked pointer into which the data will be merged. The message
-   * should be the correct type.
-   * @return returns true if the parsing was successful.
-   */
-  bool Parse(folly::IOBuf *buf, google::protobuf::Message *out);
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/BUCK b/hbase-native-client/src/hbase/client/BUCK
new file mode 100644
index 0000000..1a8f434
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/BUCK
@@ -0,0 +1,301 @@
+##
+# 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.
+
+# This is the main library.
+cxx_library(
+    name="client",
+    srcs=[
+        "async-client-scanner.cc",
+        "async-connection.cc",
+        "async-rpc-retrying-caller-factory.cc",
+        "async-rpc-retrying-caller.cc",
+        "async-scan-rpc-retrying-caller.cc",
+        "async-table-result-scanner.cc",
+        "cell.cc",
+        "client.cc",
+        "hbase-rpc-controller.cc",
+        "keyvalue-codec.cc",
+        "location-cache.cc",
+        "meta-utils.cc",
+        "increment.cc",
+        "get.cc",
+        "mutation.cc",
+        "put.cc",
+        "delete.cc",
+        "scan.cc",
+        "append.cc",
+        "scan-result-cache.cc",
+        "raw-async-table.cc",
+        "result.cc",
+        "request-converter.cc",
+        "response-converter.cc",
+        "table.cc",
+        "time-range.cc",
+        "zk-util.cc",
+        "multi-response.cc",
+        "region-result.cc",
+        "async-batch-rpc-retrying-caller.cc",
+    ],
+    deps=[
+        "//include/hbase/client:client",
+        "//src/hbase/exceptions:exceptions",
+        "//src/hbase/utils:utils",
+        "//src/hbase/connection:connection",
+        "//src/hbase/client:conf",
+        "//src/hbase/if:if",
+        "//src/hbase/serde:serde",
+        "//third-party:folly",
+        "//third-party:wangle",
+        "//third-party:zookeeper_mt",
+    ],
+    compiler_flags=['-Weffc++', '-ggdb'],
+    visibility=[
+        'PUBLIC',
+    ],)
+cxx_library(
+    name="conf",
+    exported_headers=[
+    ],
+    srcs=[
+        "configuration.cc",
+        "hbase-configuration-loader.cc",
+    ],
+    deps=["//include/hbase/client:conf", "//src/hbase/utils:utils", "//third-party:folly"],
+    compiler_flags=['-Weffc++', '-ggdb'],
+    visibility=[
+        'PUBLIC',
+    ],)
+cxx_test(
+    name="location-cache-test",
+    srcs=[
+        "location-cache-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/test-util:test-util",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="location-cache-retry-test",
+    srcs=[
+        "location-cache-retry-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/if:if",
+        "//src/hbase/serde:serde",
+        "//src/hbase/test-util:test-util",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="cell-test",
+    srcs=[
+        "cell-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="filter-test",
+    srcs=[
+        "filter-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/if:if",
+        "//src/hbase/serde:serde",
+        "//src/hbase/test-util:test-util",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="get-test",
+    srcs=[
+        "get-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="delete-test",
+    srcs=[
+        "delete-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="increment-test",
+    srcs=[
+        "increment-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="put-test",
+    srcs=[
+        "put-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="append-test",
+    srcs=[
+        "append-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="retry-test",
+    srcs=[
+        "async-rpc-retrying-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/test-util:test-util",
+        "//src/hbase/exceptions:exceptions",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="time-range-test",
+    srcs=[
+        "time-range-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="configuration-test",
+    srcs=[
+        "configuration-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="hbase-configuration-test",
+    srcs=[
+        "hbase-configuration-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="scan-test",
+    srcs=[
+        "scan-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="result-test",
+    srcs=[
+        "result-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="request-converter-test",
+    srcs=[
+        "request-converter-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/connection:connection",
+        "//src/hbase/if:if",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="client-test",
+    srcs=[
+        "client-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/if:if",
+        "//src/hbase/serde:serde",
+        "//src/hbase/test-util:test-util",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="scan-result-cache-test",
+    srcs=[
+        "scan-result-cache-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="scanner-test",
+    srcs=[
+        "scanner-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/if:if",
+        "//src/hbase/serde:serde",
+        "//src/hbase/test-util:test-util",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="zk-util-test",
+    srcs=[
+        "zk-util-test.cc",
+    ],
+    deps=[
+        ":client",
+    ],
+    run_test_separately=True,)
+cxx_test(
+    name="multi-retry-test",
+    srcs=[
+        "async-batch-rpc-retrying-test.cc",
+    ],
+    deps=[
+        ":client",
+        "//src/hbase/test-util:test-util",
+        "//src/hbase/exceptions:exceptions",
+    ],
+    run_test_separately=True,)
+cxx_binary(
+    name="simple-client",
+    srcs=[
+        "simple-client.cc",
+    ],
+    deps=[":client", "//src/hbase/connection:connection"],)
+cxx_binary(
+    name="load-client",
+    srcs=[
+        "load-client.cc",
+    ],
+    deps=[":client", "//src/hbase/connection:connection"],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/append-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/append-test.cc b/hbase-native-client/src/hbase/client/append-test.cc
new file mode 100644
index 0000000..1af138d
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/append-test.cc
@@ -0,0 +1,105 @@
+/*
+ * 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 <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "hbase/client/append.h"
+#include "hbase/client/mutation.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Append;
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Mutation;
+using hbase::TimeUtil;
+
+const constexpr int64_t Mutation::kLatestTimestamp;
+
+TEST(Append, Row) {
+  Append append{"foo"};
+  EXPECT_EQ("foo", append.row());
+}
+
+TEST(Append, Durability) {
+  Append append{"row"};
+  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, append.Durability());
+
+  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
+  append.SetDurability(skipWal);
+  EXPECT_EQ(skipWal, append.Durability());
+}
+
+TEST(Append, Timestamp) {
+  Append append{"row"};
+
+  // test default timestamp
+  EXPECT_EQ(Mutation::kLatestTimestamp, append.TimeStamp());
+
+  // set custom timestamp
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  append.SetTimeStamp(ts);
+  EXPECT_EQ(ts, append.TimeStamp());
+
+  // Add a column with custom timestamp
+  append.Add("f", "q", "v");
+  auto &cell = append.FamilyMap().at("f")[0];
+  EXPECT_EQ(ts, cell->Timestamp());
+}
+
+TEST(Append, HasFamilies) {
+  Append append{"row"};
+
+  EXPECT_EQ(false, append.HasFamilies());
+
+  append.Add("f", "q", "v");
+  EXPECT_EQ(true, append.HasFamilies());
+}
+
+TEST(Append, Add) {
+  CellType cell_type = CellType::PUT;
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+
+  // add first cell
+  Append append{"row"};
+  append.Add(std::move(cell));
+  EXPECT_EQ(1, append.FamilyMap().size());
+  EXPECT_EQ(1, append.FamilyMap().at(family).size());
+
+  // add a non-matching row
+  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+  Append append2{"foo"};
+  ASSERT_THROW(append2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
+
+  // add a second cell with same family
+  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
+  append.Add(std::move(cell3));
+  EXPECT_EQ(1, append.FamilyMap().size());
+  EXPECT_EQ(2, append.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
+  append.Add(std::move(cell4));
+  EXPECT_EQ(2, append.FamilyMap().size());
+  EXPECT_EQ(1, append.FamilyMap().at("family-2").size());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/append.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/append.cc b/hbase-native-client/src/hbase/client/append.cc
new file mode 100644
index 0000000..431a398
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/append.cc
@@ -0,0 +1,53 @@
+
+
+/*
+ * 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 "hbase/client/append.h"
+#include <folly/Conv.h>
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+#include <utility>
+
+namespace hbase {
+
+/**
+ *  @brief Append to the column from the specific family with the specified qualifier
+ *  @param family family name
+ *  @param qualifier column qualifier
+ *  @param value value to append
+ */
+Append& Append::Add(const std::string& family, const std::string& qualifier,
+                    const std::string& value) {
+  family_map_[family].push_back(std::move(
+      std::make_unique<Cell>(row_, family, qualifier, timestamp_, value, hbase::CellType::PUT)));
+  return *this;
+}
+Append& Append::Add(std::unique_ptr<Cell> cell) {
+  if (cell->Row() != row_) {
+    throw std::runtime_error("The row in " + cell->DebugString() +
+                             " doesn't match the original one " + row_);
+  }
+
+  family_map_[cell->Family()].push_back(std::move(cell));
+  return *this;
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-caller.cc b/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-caller.cc
new file mode 100644
index 0000000..6699b90
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/async-batch-rpc-retrying-caller.cc
@@ -0,0 +1,488 @@
+/*
+ * 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 "hbase/client/async-batch-rpc-retrying-caller.h"
+#include <glog/logging.h>
+#include <limits>
+
+using folly::Future;
+using folly::Promise;
+using folly::Try;
+using hbase::pb::ServerName;
+using hbase::pb::TableName;
+using hbase::security::User;
+using std::chrono::nanoseconds;
+using std::chrono::milliseconds;
+
+namespace hbase {
+
+template <typename REQ, typename RESP>
+AsyncBatchRpcRetryingCaller<REQ, RESP>::AsyncBatchRpcRetryingCaller(
+    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
+    std::shared_ptr<TableName> table_name, const std::vector<REQ> &actions, nanoseconds pause_ns,
+    int32_t max_attempts, nanoseconds operation_timeout_ns, nanoseconds rpc_timeout_ns,
+    int32_t start_log_errors_count)
+    : conn_(conn),
+      retry_timer_(retry_timer),
+      table_name_(table_name),
+      pause_ns_(pause_ns),
+      operation_timeout_ns_(operation_timeout_ns),
+      rpc_timeout_ns_(rpc_timeout_ns),
+      start_log_errors_count_(start_log_errors_count) {
+  CHECK(conn_ != nullptr);
+  CHECK(retry_timer_ != nullptr);
+  location_cache_ = conn_->region_locator();
+  rpc_client_ = conn_->rpc_client();
+  cpu_pool_ = conn_->cpu_executor();
+  CHECK(location_cache_ != nullptr);
+  CHECK(rpc_client_ != nullptr);
+  CHECK(cpu_pool_ != nullptr);
+
+  max_attempts_ = ConnectionUtils::Retries2Attempts(max_attempts);
+  uint32_t index = 0;
+  for (auto row : actions) {
+    actions_.push_back(std::make_shared<Action>(row, index));
+    Promise<RESP> prom{};
+    action2promises_.insert(std::pair<uint64_t, Promise<RESP>>(index, std::move(prom)));
+    action2futures_.push_back(action2promises_[index++].getFuture());
+  }
+}
+
+template <typename REQ, typename RESP>
+AsyncBatchRpcRetryingCaller<REQ, RESP>::~AsyncBatchRpcRetryingCaller() {}
+
+template <typename REQ, typename RESP>
+Future<std::vector<Try<RESP>>> AsyncBatchRpcRetryingCaller<REQ, RESP>::Call() {
+  GroupAndSend(actions_, 1);
+  return collectAll(action2futures_);
+}
+
+template <typename REQ, typename RESP>
+int64_t AsyncBatchRpcRetryingCaller<REQ, RESP>::RemainingTimeNs() {
+  return operation_timeout_ns_.count() - (TimeUtil::GetNowNanos() - start_ns_);
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::LogException(
+    int32_t tries, std::shared_ptr<RegionRequest> region_request,
+    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
+  if (tries > start_log_errors_count_) {
+    std::string regions;
+    regions += region_request->region_location()->region_name() + ", ";
+    LOG(WARNING) << "Process batch for " << regions << " in " << table_name_->namespace_() << ":"
+                 << table_name_->qualifier() << " from " << server_name->host_name()
+                 << " failed, tries=" << tries << ":- " << ew.what().toStdString();
+  }
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::LogException(
+    int32_t tries, const std::vector<std::shared_ptr<RegionRequest>> &region_requests,
+    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
+  if (tries > start_log_errors_count_) {
+    std::string regions;
+    for (const auto region_request : region_requests) {
+      regions += region_request->region_location()->region_name() + ", ";
+    }
+    LOG(WARNING) << "Process batch for " << regions << " in " << table_name_->namespace_() << ":"
+                 << table_name_->qualifier() << " from " << server_name->host_name()
+                 << " failed, tries=" << tries << ew.what().toStdString();
+  }
+}
+
+template <typename REQ, typename RESP>
+const std::string AsyncBatchRpcRetryingCaller<REQ, RESP>::GetExtraContextForError(
+    std::shared_ptr<ServerName> server_name) {
+  return server_name ? server_name->ShortDebugString() : "";
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddError(const std::shared_ptr<Action> &action,
+                                                      const folly::exception_wrapper &ew,
+                                                      std::shared_ptr<ServerName> server_name) {
+  ThrowableWithExtraContext twec(ew, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
+  AddAction2Error(action->original_index(), twec);
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddError(
+    const std::vector<std::shared_ptr<Action>> &actions, const folly::exception_wrapper &ew,
+    std::shared_ptr<ServerName> server_name) {
+  for (const auto action : actions) {
+    AddError(action, ew, server_name);
+  }
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailOne(const std::shared_ptr<Action> &action,
+                                                     int32_t tries,
+                                                     const folly::exception_wrapper &ew,
+                                                     int64_t current_time,
+                                                     const std::string extras) {
+  auto action_index = action->original_index();
+  auto itr = action2promises_.find(action_index);
+  if (itr != action2promises_.end()) {
+    if (itr->second.isFulfilled()) {
+      return;
+    }
+  }
+  ThrowableWithExtraContext twec(ew, current_time, extras);
+  AddAction2Error(action_index, twec);
+  action2promises_[action_index].setException(
+      RetriesExhaustedException(tries - 1, action2errors_[action_index]));
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailAll(
+    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries,
+    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
+  for (const auto action : actions) {
+    FailOne(action, tries, ew, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
+  }
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailAll(
+    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
+  for (const auto action : actions) {
+    auto action_index = action->original_index();
+    auto itr = action2promises_.find(action_index);
+    if (itr->second.isFulfilled()) {
+      return;
+    }
+    action2promises_[action_index].setException(
+        RetriesExhaustedException(tries, action2errors_[action_index]));
+  }
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddAction2Error(
+    uint64_t action_index, const ThrowableWithExtraContext &twec) {
+  auto erritr = action2errors_.find(action_index);
+  if (erritr != action2errors_.end()) {
+    erritr->second->push_back(twec);
+  } else {
+    action2errors_[action_index] = std::make_shared<std::vector<ThrowableWithExtraContext>>();
+    action2errors_[action_index]->push_back(twec);
+  }
+  return;
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnError(const ActionsByRegion &actions_by_region,
+                                                     int32_t tries,
+                                                     const folly::exception_wrapper &ew,
+                                                     std::shared_ptr<ServerName> server_name) {
+  std::vector<std::shared_ptr<Action>> copied_actions;
+  std::vector<std::shared_ptr<RegionRequest>> region_requests;
+  for (const auto &action_by_region : actions_by_region) {
+    region_requests.push_back(action_by_region.second);
+    for (const auto &action : action_by_region.second->actions()) {
+      copied_actions.push_back(action);
+    }
+  }
+
+  LogException(tries, region_requests, ew, server_name);
+  if ((tries >= max_attempts_) || !ExceptionUtil::ShouldRetry(ew)) {
+    FailAll(copied_actions, tries, ew, server_name);
+    return;
+  }
+  AddError(copied_actions, ew, server_name);
+  TryResubmit(copied_actions, tries);
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::TryResubmit(
+    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
+  int64_t delay_ns;
+  if (operation_timeout_ns_.count() > 0) {
+    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
+    if (max_delay_ns <= 0) {
+      FailAll(actions, tries);
+      return;
+    }
+    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_ns_.count(), tries - 1));
+  } else {
+    delay_ns = ConnectionUtils::GetPauseTime(pause_ns_.count(), tries - 1);
+  }
+
+  conn_->retry_executor()->add([=]() {
+    retry_timer_->scheduleTimeoutFn(
+        [=]() { conn_->cpu_executor()->add([=]() { GroupAndSend(actions, tries + 1); }); },
+        milliseconds(TimeUtil::ToMillis(delay_ns)));
+  });
+}
+
+template <typename REQ, typename RESP>
+Future<std::vector<Try<std::shared_ptr<RegionLocation>>>>
+AsyncBatchRpcRetryingCaller<REQ, RESP>::GetRegionLocations(
+    const std::vector<std::shared_ptr<Action>> &actions, int64_t locate_timeout_ns) {
+  auto locs = std::vector<Future<std::shared_ptr<RegionLocation>>>{};
+  for (auto const &action : actions) {
+    locs.push_back(location_cache_->LocateRegion(*table_name_, action->action()->row(),
+                                                 RegionLocateType::kCurrent, locate_timeout_ns));
+  }
+
+  return collectAll(locs);
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::GroupAndSend(
+    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
+  int64_t locate_timeout_ns;
+  if (operation_timeout_ns_.count() > 0) {
+    locate_timeout_ns = RemainingTimeNs();
+    if (locate_timeout_ns <= 0) {
+      FailAll(actions, tries);
+      return;
+    }
+  } else {
+    locate_timeout_ns = -1L;
+  }
+
+  GetRegionLocations(actions, locate_timeout_ns)
+      .then([=](std::vector<Try<std::shared_ptr<RegionLocation>>> &loc) {
+        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
+        ActionsByServer actions_by_server;
+        std::vector<std::shared_ptr<Action>> locate_failed;
+
+        for (uint64_t i = 0; i < loc.size(); ++i) {
+          auto action = actions[i];
+          if (loc[i].hasValue()) {
+            auto region_loc = loc[i].value();
+            // Add it to actions_by_server;
+            auto search =
+                actions_by_server.find(std::make_shared<ServerName>(region_loc->server_name()));
+            if (search != actions_by_server.end()) {
+              search->second->AddActionsByRegion(region_loc, action);
+            } else {
+              auto server_request = std::make_shared<ServerRequest>(region_loc);
+              server_request->AddActionsByRegion(region_loc, action);
+              auto server_name = std::make_shared<ServerName>(region_loc->server_name());
+              actions_by_server[server_name] = server_request;
+            }
+            VLOG(5) << "rowkey [" << action->action()->row() << "] of table["
+                    << table_name_->ShortDebugString() << "] found in ["
+                    << region_loc->region_name() << "]; RS["
+                    << region_loc->server_name().host_name() << ":"
+                    << region_loc->server_name().port() << "];";
+          } else if (loc[i].hasException()) {
+            folly::exception_wrapper ew = loc[i].exception();
+            VLOG(1) << "GetRegionLocations() exception: " << ew.what().toStdString()
+                    << "for index:" << i << "; tries: " << tries
+                    << "; max_attempts_: " << max_attempts_;
+            // We might receive runtime error from location-cache.cc too, we are doing FailOne and
+            // continue next one
+            if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(ew)) {
+              FailOne(action, tries, ew, TimeUtil::GetNowNanos(), ew.what().toStdString());
+            } else {
+              AddError(action, loc[i].exception(), nullptr);
+              locate_failed.push_back(action);
+            }
+          }
+        }
+        if (!actions_by_server.empty()) {
+          Send(actions_by_server, tries);
+        }
+
+        if (!locate_failed.empty()) {
+          TryResubmit(locate_failed, tries);
+        }
+      })
+      .onError([=](const folly::exception_wrapper &ew) {
+        VLOG(1) << "GetRegionLocations() exception: " << ew.what().toStdString()
+                << "tries: " << tries << "; max_attempts_: " << max_attempts_;
+        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
+        if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(ew)) {
+          FailAll(actions, tries, ew, nullptr);
+        } else {
+          TryResubmit(actions, tries);
+        }
+      });
+  return;
+}
+
+template <typename REQ, typename RESP>
+Future<std::vector<Try<std::unique_ptr<Response>>>>
+AsyncBatchRpcRetryingCaller<REQ, RESP>::GetMultiResponse(const ActionsByServer &actions_by_server) {
+  auto multi_calls = std::vector<Future<std::unique_ptr<hbase::Response>>>{};
+  auto user = User::defaultUser();
+  for (const auto &action_by_server : actions_by_server) {
+    std::unique_ptr<Request> multi_req =
+        RequestConverter::ToMultiRequest(action_by_server.second->actions_by_region());
+    auto host = action_by_server.first->host_name();
+    int port = action_by_server.first->port();
+    multi_calls.push_back(
+        rpc_client_->AsyncCall(host, port, std::move(multi_req), user, "ClientService"));
+  }
+  return collectAll(multi_calls);
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::Send(const ActionsByServer &actions_by_server,
+                                                  int32_t tries) {
+  int64_t remaining_ns;
+  if (operation_timeout_ns_.count() > 0) {
+    remaining_ns = RemainingTimeNs();
+    if (remaining_ns <= 0) {
+      std::vector<std::shared_ptr<Action>> failed_actions;
+      for (const auto &action_by_server : actions_by_server) {
+        for (auto &value : action_by_server.second->actions_by_region()) {
+          for (const auto &failed_action : value.second->actions()) {
+            failed_actions.push_back(failed_action);
+          }
+        }
+      }
+      FailAll(failed_actions, tries);
+      return;
+    }
+  } else {
+    remaining_ns = std::numeric_limits<int64_t>::max();
+  }
+
+  std::vector<std::shared_ptr<Request>> multi_reqv;
+  for (const auto &action_by_server : actions_by_server)
+    multi_reqv.push_back(
+        std::move(RequestConverter::ToMultiRequest(action_by_server.second->actions_by_region())));
+
+  GetMultiResponse(actions_by_server)
+      .then([=](const std::vector<Try<std::unique_ptr<hbase::Response>>> &completed_responses) {
+        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
+        uint64_t num = 0;
+        for (const auto &action_by_server : actions_by_server) {
+          if (completed_responses[num].hasValue()) {
+            auto multi_response =
+                ResponseConverter::GetResults(multi_reqv[num], *completed_responses[num].value(),
+                                              action_by_server.second->actions_by_region());
+            OnComplete(action_by_server.second->actions_by_region(), tries, action_by_server.first,
+                       std::move(multi_response));
+          } else if (completed_responses[num].hasException()) {
+            folly::exception_wrapper ew = completed_responses[num].exception();
+            VLOG(1) << "GetMultiResponse() exception: " << ew.what().toStdString()
+                    << " from server for action index:" << num;
+            OnError(action_by_server.second->actions_by_region(), tries, ew,
+                    action_by_server.first);
+          }
+          num++;
+        }
+      })
+      .onError([=](const folly::exception_wrapper &ew) {
+        VLOG(1) << "GetMultiResponse() exception: " << ew.what().toStdString();
+        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
+        for (const auto &action_by_server : actions_by_server) {
+          OnError(action_by_server.second->actions_by_region(), tries, ew, action_by_server.first);
+        }
+      });
+  return;
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnComplete(
+    const ActionsByRegion &actions_by_region, int32_t tries,
+    const std::shared_ptr<ServerName> server_name,
+    const std::unique_ptr<hbase::MultiResponse> multi_response) {
+  std::vector<std::shared_ptr<Action>> failed_actions;
+  const auto region_results = multi_response->RegionResults();
+  for (const auto &action_by_region : actions_by_region) {
+    auto region_result_itr = region_results.find(action_by_region.first);
+    if (region_result_itr != region_results.end()) {
+      for (const auto &action : action_by_region.second->actions()) {
+        OnComplete(action, action_by_region.second, tries, server_name, region_result_itr->second,
+                   failed_actions);
+      }
+    } else if (region_result_itr == region_results.end()) {
+      auto region_exc = multi_response->RegionException(action_by_region.first);
+      if (region_exc == nullptr) {
+        // FailAll actions for this particular region as inconsistent server response. So we raise
+        // this exception to the application
+        std::string err_msg = "Invalid response: Server " + server_name->ShortDebugString() +
+                              " sent us neither results nor exceptions for " +
+                              action_by_region.first;
+        VLOG(1) << err_msg;
+        auto ew = folly::make_exception_wrapper<std::runtime_error>(err_msg);
+        FailAll(action_by_region.second->actions(), tries, ew, server_name);
+      } else {
+        // Eg: org.apache.hadoop.hbase.NotServingRegionException:
+        LogException(tries, action_by_region.second, *region_exc, server_name);
+        if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(*region_exc)) {
+          FailAll(action_by_region.second->actions(), tries, *region_exc, server_name);
+          return;
+        }
+        location_cache_->UpdateCachedLocation(*action_by_region.second->region_location(),
+                                              *region_exc);
+        AddError(action_by_region.second->actions(), *region_exc, server_name);
+        for (const auto &action : action_by_region.second->actions()) {
+          failed_actions.push_back(action);
+        }
+      }
+    }
+  }
+  if (!failed_actions.empty()) {
+    TryResubmit(failed_actions, tries);
+  }
+
+  return;
+}
+
+template <typename REQ, typename RESP>
+void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnComplete(
+    const std::shared_ptr<Action> &action, const std::shared_ptr<RegionRequest> &region_request,
+    int32_t tries, const std::shared_ptr<ServerName> &server_name,
+    const std::shared_ptr<RegionResult> &region_result,
+    std::vector<std::shared_ptr<Action>> &failed_actions) {
+  std::string err_msg;
+  try {
+    auto result_or_exc = region_result->ResultOrException(action->original_index());
+    auto result = std::get<0>(*result_or_exc);
+    auto exc = std::get<1>(*result_or_exc);
+    if (exc != nullptr) {
+      LogException(tries, region_request, *exc, server_name);
+      if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(*exc)) {
+        FailOne(action, tries, *exc, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
+      } else {
+        failed_actions.push_back(action);
+      }
+    } else if (result != nullptr) {
+      action2promises_[action->original_index()].setValue(std::move(result));
+    } else {
+      std::string err_msg = "Invalid response: Server " + server_name->ShortDebugString() +
+                            " sent us neither results nor exceptions for request @ index " +
+                            std::to_string(action->original_index()) + ", row " +
+                            action->action()->row() + " of " +
+                            region_request->region_location()->region_name();
+      VLOG(1) << err_msg;
+      auto ew = folly::make_exception_wrapper<std::runtime_error>(err_msg);
+      AddError(action, ew, server_name);
+      failed_actions.push_back(action);
+    }
+  } catch (const std::out_of_range &oor) {
+    // This should never occur. Error in logic. Throwing std::runtime_error from here. Will be
+    // retried or failed
+    std::string err_msg = "ResultOrException not present @ index " +
+                          std::to_string(action->original_index()) + ", row " +
+                          action->action()->row() + " of " +
+                          region_request->region_location()->region_name();
+    throw std::runtime_error(err_msg);
+  }
+  return;
+}
+
+template class AsyncBatchRpcRetryingCaller<std::shared_ptr<hbase::Row>,
+                                           std::shared_ptr<hbase::Result>>;
+} /* namespace hbase */


[23/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/append.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/append.h b/hbase-native-client/core/append.h
deleted file mode 100644
index cf9ac24..0000000
--- a/hbase-native-client/core/append.h
+++ /dev/null
@@ -1,56 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/cell.h"
-#include "core/mutation.h"
-
-namespace hbase {
-
-class Append : public Mutation {
- public:
-  /**
-   * Constructors
-   */
-  explicit Append(const std::string& row) : Mutation(row) {}
-  Append(const Append& cappend) : Mutation(cappend) {}
-  Append& operator=(const Append& cappend) {
-    Mutation::operator=(cappend);
-    return *this;
-  }
-
-  ~Append() = default;
-
-  /**
-   *  @brief Add the specified column and value to this Append operation.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param value value to append
-   */
-  Append& Add(const std::string& family, const std::string& qualifier, const std::string& value);
-  Append& Add(std::unique_ptr<Cell> cell);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-batch-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-batch-rpc-retrying-caller.cc b/hbase-native-client/core/async-batch-rpc-retrying-caller.cc
deleted file mode 100644
index dfbf7e7..0000000
--- a/hbase-native-client/core/async-batch-rpc-retrying-caller.cc
+++ /dev/null
@@ -1,488 +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/async-batch-rpc-retrying-caller.h"
-#include <glog/logging.h>
-#include <limits>
-
-using folly::Future;
-using folly::Promise;
-using folly::Try;
-using hbase::pb::ServerName;
-using hbase::pb::TableName;
-using hbase::security::User;
-using std::chrono::nanoseconds;
-using std::chrono::milliseconds;
-
-namespace hbase {
-
-template <typename REQ, typename RESP>
-AsyncBatchRpcRetryingCaller<REQ, RESP>::AsyncBatchRpcRetryingCaller(
-    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
-    std::shared_ptr<TableName> table_name, const std::vector<REQ> &actions, nanoseconds pause_ns,
-    int32_t max_attempts, nanoseconds operation_timeout_ns, nanoseconds rpc_timeout_ns,
-    int32_t start_log_errors_count)
-    : conn_(conn),
-      retry_timer_(retry_timer),
-      table_name_(table_name),
-      pause_ns_(pause_ns),
-      operation_timeout_ns_(operation_timeout_ns),
-      rpc_timeout_ns_(rpc_timeout_ns),
-      start_log_errors_count_(start_log_errors_count) {
-  CHECK(conn_ != nullptr);
-  CHECK(retry_timer_ != nullptr);
-  location_cache_ = conn_->region_locator();
-  rpc_client_ = conn_->rpc_client();
-  cpu_pool_ = conn_->cpu_executor();
-  CHECK(location_cache_ != nullptr);
-  CHECK(rpc_client_ != nullptr);
-  CHECK(cpu_pool_ != nullptr);
-
-  max_attempts_ = ConnectionUtils::Retries2Attempts(max_attempts);
-  uint32_t index = 0;
-  for (auto row : actions) {
-    actions_.push_back(std::make_shared<Action>(row, index));
-    Promise<RESP> prom{};
-    action2promises_.insert(std::pair<uint64_t, Promise<RESP>>(index, std::move(prom)));
-    action2futures_.push_back(action2promises_[index++].getFuture());
-  }
-}
-
-template <typename REQ, typename RESP>
-AsyncBatchRpcRetryingCaller<REQ, RESP>::~AsyncBatchRpcRetryingCaller() {}
-
-template <typename REQ, typename RESP>
-Future<std::vector<Try<RESP>>> AsyncBatchRpcRetryingCaller<REQ, RESP>::Call() {
-  GroupAndSend(actions_, 1);
-  return collectAll(action2futures_);
-}
-
-template <typename REQ, typename RESP>
-int64_t AsyncBatchRpcRetryingCaller<REQ, RESP>::RemainingTimeNs() {
-  return operation_timeout_ns_.count() - (TimeUtil::GetNowNanos() - start_ns_);
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::LogException(
-    int32_t tries, std::shared_ptr<RegionRequest> region_request,
-    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
-  if (tries > start_log_errors_count_) {
-    std::string regions;
-    regions += region_request->region_location()->region_name() + ", ";
-    LOG(WARNING) << "Process batch for " << regions << " in " << table_name_->namespace_() << ":"
-                 << table_name_->qualifier() << " from " << server_name->host_name()
-                 << " failed, tries=" << tries << ":- " << ew.what().toStdString();
-  }
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::LogException(
-    int32_t tries, const std::vector<std::shared_ptr<RegionRequest>> &region_requests,
-    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
-  if (tries > start_log_errors_count_) {
-    std::string regions;
-    for (const auto region_request : region_requests) {
-      regions += region_request->region_location()->region_name() + ", ";
-    }
-    LOG(WARNING) << "Process batch for " << regions << " in " << table_name_->namespace_() << ":"
-                 << table_name_->qualifier() << " from " << server_name->host_name()
-                 << " failed, tries=" << tries << ew.what().toStdString();
-  }
-}
-
-template <typename REQ, typename RESP>
-const std::string AsyncBatchRpcRetryingCaller<REQ, RESP>::GetExtraContextForError(
-    std::shared_ptr<ServerName> server_name) {
-  return server_name ? server_name->ShortDebugString() : "";
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddError(const std::shared_ptr<Action> &action,
-                                                      const folly::exception_wrapper &ew,
-                                                      std::shared_ptr<ServerName> server_name) {
-  ThrowableWithExtraContext twec(ew, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
-  AddAction2Error(action->original_index(), twec);
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddError(
-    const std::vector<std::shared_ptr<Action>> &actions, const folly::exception_wrapper &ew,
-    std::shared_ptr<ServerName> server_name) {
-  for (const auto action : actions) {
-    AddError(action, ew, server_name);
-  }
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailOne(const std::shared_ptr<Action> &action,
-                                                     int32_t tries,
-                                                     const folly::exception_wrapper &ew,
-                                                     int64_t current_time,
-                                                     const std::string extras) {
-  auto action_index = action->original_index();
-  auto itr = action2promises_.find(action_index);
-  if (itr != action2promises_.end()) {
-    if (itr->second.isFulfilled()) {
-      return;
-    }
-  }
-  ThrowableWithExtraContext twec(ew, current_time, extras);
-  AddAction2Error(action_index, twec);
-  action2promises_[action_index].setException(
-      RetriesExhaustedException(tries - 1, action2errors_[action_index]));
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailAll(
-    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries,
-    const folly::exception_wrapper &ew, std::shared_ptr<ServerName> server_name) {
-  for (const auto action : actions) {
-    FailOne(action, tries, ew, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
-  }
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::FailAll(
-    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
-  for (const auto action : actions) {
-    auto action_index = action->original_index();
-    auto itr = action2promises_.find(action_index);
-    if (itr->second.isFulfilled()) {
-      return;
-    }
-    action2promises_[action_index].setException(
-        RetriesExhaustedException(tries, action2errors_[action_index]));
-  }
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::AddAction2Error(
-    uint64_t action_index, const ThrowableWithExtraContext &twec) {
-  auto erritr = action2errors_.find(action_index);
-  if (erritr != action2errors_.end()) {
-    erritr->second->push_back(twec);
-  } else {
-    action2errors_[action_index] = std::make_shared<std::vector<ThrowableWithExtraContext>>();
-    action2errors_[action_index]->push_back(twec);
-  }
-  return;
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnError(const ActionsByRegion &actions_by_region,
-                                                     int32_t tries,
-                                                     const folly::exception_wrapper &ew,
-                                                     std::shared_ptr<ServerName> server_name) {
-  std::vector<std::shared_ptr<Action>> copied_actions;
-  std::vector<std::shared_ptr<RegionRequest>> region_requests;
-  for (const auto &action_by_region : actions_by_region) {
-    region_requests.push_back(action_by_region.second);
-    for (const auto &action : action_by_region.second->actions()) {
-      copied_actions.push_back(action);
-    }
-  }
-
-  LogException(tries, region_requests, ew, server_name);
-  if ((tries >= max_attempts_) || !ExceptionUtil::ShouldRetry(ew)) {
-    FailAll(copied_actions, tries, ew, server_name);
-    return;
-  }
-  AddError(copied_actions, ew, server_name);
-  TryResubmit(copied_actions, tries);
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::TryResubmit(
-    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
-  int64_t delay_ns;
-  if (operation_timeout_ns_.count() > 0) {
-    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
-    if (max_delay_ns <= 0) {
-      FailAll(actions, tries);
-      return;
-    }
-    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_ns_.count(), tries - 1));
-  } else {
-    delay_ns = ConnectionUtils::GetPauseTime(pause_ns_.count(), tries - 1);
-  }
-
-  conn_->retry_executor()->add([=]() {
-    retry_timer_->scheduleTimeoutFn(
-        [=]() { conn_->cpu_executor()->add([=]() { GroupAndSend(actions, tries + 1); }); },
-        milliseconds(TimeUtil::ToMillis(delay_ns)));
-  });
-}
-
-template <typename REQ, typename RESP>
-Future<std::vector<Try<std::shared_ptr<RegionLocation>>>>
-AsyncBatchRpcRetryingCaller<REQ, RESP>::GetRegionLocations(
-    const std::vector<std::shared_ptr<Action>> &actions, int64_t locate_timeout_ns) {
-  auto locs = std::vector<Future<std::shared_ptr<RegionLocation>>>{};
-  for (auto const &action : actions) {
-    locs.push_back(location_cache_->LocateRegion(*table_name_, action->action()->row(),
-                                                 RegionLocateType::kCurrent, locate_timeout_ns));
-  }
-
-  return collectAll(locs);
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::GroupAndSend(
-    const std::vector<std::shared_ptr<Action>> &actions, int32_t tries) {
-  int64_t locate_timeout_ns;
-  if (operation_timeout_ns_.count() > 0) {
-    locate_timeout_ns = RemainingTimeNs();
-    if (locate_timeout_ns <= 0) {
-      FailAll(actions, tries);
-      return;
-    }
-  } else {
-    locate_timeout_ns = -1L;
-  }
-
-  GetRegionLocations(actions, locate_timeout_ns)
-      .then([=](std::vector<Try<std::shared_ptr<RegionLocation>>> &loc) {
-        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
-        ActionsByServer actions_by_server;
-        std::vector<std::shared_ptr<Action>> locate_failed;
-
-        for (uint64_t i = 0; i < loc.size(); ++i) {
-          auto action = actions[i];
-          if (loc[i].hasValue()) {
-            auto region_loc = loc[i].value();
-            // Add it to actions_by_server;
-            auto search =
-                actions_by_server.find(std::make_shared<ServerName>(region_loc->server_name()));
-            if (search != actions_by_server.end()) {
-              search->second->AddActionsByRegion(region_loc, action);
-            } else {
-              auto server_request = std::make_shared<ServerRequest>(region_loc);
-              server_request->AddActionsByRegion(region_loc, action);
-              auto server_name = std::make_shared<ServerName>(region_loc->server_name());
-              actions_by_server[server_name] = server_request;
-            }
-            VLOG(5) << "rowkey [" << action->action()->row() << "] of table["
-                    << table_name_->ShortDebugString() << "] found in ["
-                    << region_loc->region_name() << "]; RS["
-                    << region_loc->server_name().host_name() << ":"
-                    << region_loc->server_name().port() << "];";
-          } else if (loc[i].hasException()) {
-            folly::exception_wrapper ew = loc[i].exception();
-            VLOG(1) << "GetRegionLocations() exception: " << ew.what().toStdString()
-                    << "for index:" << i << "; tries: " << tries
-                    << "; max_attempts_: " << max_attempts_;
-            // We might receive runtime error from location-cache.cc too, we are doing FailOne and
-            // continue next one
-            if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(ew)) {
-              FailOne(action, tries, ew, TimeUtil::GetNowNanos(), ew.what().toStdString());
-            } else {
-              AddError(action, loc[i].exception(), nullptr);
-              locate_failed.push_back(action);
-            }
-          }
-        }
-        if (!actions_by_server.empty()) {
-          Send(actions_by_server, tries);
-        }
-
-        if (!locate_failed.empty()) {
-          TryResubmit(locate_failed, tries);
-        }
-      })
-      .onError([=](const folly::exception_wrapper &ew) {
-        VLOG(1) << "GetRegionLocations() exception: " << ew.what().toStdString()
-                << "tries: " << tries << "; max_attempts_: " << max_attempts_;
-        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
-        if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(ew)) {
-          FailAll(actions, tries, ew, nullptr);
-        } else {
-          TryResubmit(actions, tries);
-        }
-      });
-  return;
-}
-
-template <typename REQ, typename RESP>
-Future<std::vector<Try<std::unique_ptr<Response>>>>
-AsyncBatchRpcRetryingCaller<REQ, RESP>::GetMultiResponse(const ActionsByServer &actions_by_server) {
-  auto multi_calls = std::vector<Future<std::unique_ptr<hbase::Response>>>{};
-  auto user = User::defaultUser();
-  for (const auto &action_by_server : actions_by_server) {
-    std::unique_ptr<Request> multi_req =
-        RequestConverter::ToMultiRequest(action_by_server.second->actions_by_region());
-    auto host = action_by_server.first->host_name();
-    int port = action_by_server.first->port();
-    multi_calls.push_back(
-        rpc_client_->AsyncCall(host, port, std::move(multi_req), user, "ClientService"));
-  }
-  return collectAll(multi_calls);
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::Send(const ActionsByServer &actions_by_server,
-                                                  int32_t tries) {
-  int64_t remaining_ns;
-  if (operation_timeout_ns_.count() > 0) {
-    remaining_ns = RemainingTimeNs();
-    if (remaining_ns <= 0) {
-      std::vector<std::shared_ptr<Action>> failed_actions;
-      for (const auto &action_by_server : actions_by_server) {
-        for (auto &value : action_by_server.second->actions_by_region()) {
-          for (const auto &failed_action : value.second->actions()) {
-            failed_actions.push_back(failed_action);
-          }
-        }
-      }
-      FailAll(failed_actions, tries);
-      return;
-    }
-  } else {
-    remaining_ns = std::numeric_limits<int64_t>::max();
-  }
-
-  std::vector<std::shared_ptr<Request>> multi_reqv;
-  for (const auto &action_by_server : actions_by_server)
-    multi_reqv.push_back(
-        std::move(RequestConverter::ToMultiRequest(action_by_server.second->actions_by_region())));
-
-  GetMultiResponse(actions_by_server)
-      .then([=](const std::vector<Try<std::unique_ptr<hbase::Response>>> &completed_responses) {
-        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
-        uint64_t num = 0;
-        for (const auto &action_by_server : actions_by_server) {
-          if (completed_responses[num].hasValue()) {
-            auto multi_response =
-                ResponseConverter::GetResults(multi_reqv[num], *completed_responses[num].value(),
-                                              action_by_server.second->actions_by_region());
-            OnComplete(action_by_server.second->actions_by_region(), tries, action_by_server.first,
-                       std::move(multi_response));
-          } else if (completed_responses[num].hasException()) {
-            folly::exception_wrapper ew = completed_responses[num].exception();
-            VLOG(1) << "GetMultiResponse() exception: " << ew.what().toStdString()
-                    << " from server for action index:" << num;
-            OnError(action_by_server.second->actions_by_region(), tries, ew,
-                    action_by_server.first);
-          }
-          num++;
-        }
-      })
-      .onError([=](const folly::exception_wrapper &ew) {
-        VLOG(1) << "GetMultiResponse() exception: " << ew.what().toStdString();
-        std::lock_guard<std::recursive_mutex> lck(multi_mutex_);
-        for (const auto &action_by_server : actions_by_server) {
-          OnError(action_by_server.second->actions_by_region(), tries, ew, action_by_server.first);
-        }
-      });
-  return;
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnComplete(
-    const ActionsByRegion &actions_by_region, int32_t tries,
-    const std::shared_ptr<ServerName> server_name,
-    const std::unique_ptr<hbase::MultiResponse> multi_response) {
-  std::vector<std::shared_ptr<Action>> failed_actions;
-  const auto region_results = multi_response->RegionResults();
-  for (const auto &action_by_region : actions_by_region) {
-    auto region_result_itr = region_results.find(action_by_region.first);
-    if (region_result_itr != region_results.end()) {
-      for (const auto &action : action_by_region.second->actions()) {
-        OnComplete(action, action_by_region.second, tries, server_name, region_result_itr->second,
-                   failed_actions);
-      }
-    } else if (region_result_itr == region_results.end()) {
-      auto region_exc = multi_response->RegionException(action_by_region.first);
-      if (region_exc == nullptr) {
-        // FailAll actions for this particular region as inconsistent server response. So we raise
-        // this exception to the application
-        std::string err_msg = "Invalid response: Server " + server_name->ShortDebugString() +
-                              " sent us neither results nor exceptions for " +
-                              action_by_region.first;
-        VLOG(1) << err_msg;
-        auto ew = folly::make_exception_wrapper<std::runtime_error>(err_msg);
-        FailAll(action_by_region.second->actions(), tries, ew, server_name);
-      } else {
-        // Eg: org.apache.hadoop.hbase.NotServingRegionException:
-        LogException(tries, action_by_region.second, *region_exc, server_name);
-        if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(*region_exc)) {
-          FailAll(action_by_region.second->actions(), tries, *region_exc, server_name);
-          return;
-        }
-        location_cache_->UpdateCachedLocation(*action_by_region.second->region_location(),
-                                              *region_exc);
-        AddError(action_by_region.second->actions(), *region_exc, server_name);
-        for (const auto &action : action_by_region.second->actions()) {
-          failed_actions.push_back(action);
-        }
-      }
-    }
-  }
-  if (!failed_actions.empty()) {
-    TryResubmit(failed_actions, tries);
-  }
-
-  return;
-}
-
-template <typename REQ, typename RESP>
-void AsyncBatchRpcRetryingCaller<REQ, RESP>::OnComplete(
-    const std::shared_ptr<Action> &action, const std::shared_ptr<RegionRequest> &region_request,
-    int32_t tries, const std::shared_ptr<ServerName> &server_name,
-    const std::shared_ptr<RegionResult> &region_result,
-    std::vector<std::shared_ptr<Action>> &failed_actions) {
-  std::string err_msg;
-  try {
-    auto result_or_exc = region_result->ResultOrException(action->original_index());
-    auto result = std::get<0>(*result_or_exc);
-    auto exc = std::get<1>(*result_or_exc);
-    if (exc != nullptr) {
-      LogException(tries, region_request, *exc, server_name);
-      if (tries >= max_attempts_ || !ExceptionUtil::ShouldRetry(*exc)) {
-        FailOne(action, tries, *exc, TimeUtil::GetNowNanos(), GetExtraContextForError(server_name));
-      } else {
-        failed_actions.push_back(action);
-      }
-    } else if (result != nullptr) {
-      action2promises_[action->original_index()].setValue(std::move(result));
-    } else {
-      std::string err_msg = "Invalid response: Server " + server_name->ShortDebugString() +
-                            " sent us neither results nor exceptions for request @ index " +
-                            std::to_string(action->original_index()) + ", row " +
-                            action->action()->row() + " of " +
-                            region_request->region_location()->region_name();
-      VLOG(1) << err_msg;
-      auto ew = folly::make_exception_wrapper<std::runtime_error>(err_msg);
-      AddError(action, ew, server_name);
-      failed_actions.push_back(action);
-    }
-  } catch (const std::out_of_range &oor) {
-    // This should never occur. Error in logic. Throwing std::runtime_error from here. Will be
-    // retried or failed
-    std::string err_msg = "ResultOrException not present @ index " +
-                          std::to_string(action->original_index()) + ", row " +
-                          action->action()->row() + " of " +
-                          region_request->region_location()->region_name();
-    throw std::runtime_error(err_msg);
-  }
-  return;
-}
-
-template class AsyncBatchRpcRetryingCaller<std::shared_ptr<hbase::Row>,
-                                           std::shared_ptr<hbase::Result>>;
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-batch-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-batch-rpc-retrying-caller.h b/hbase-native-client/core/async-batch-rpc-retrying-caller.h
deleted file mode 100644
index 9194b04..0000000
--- a/hbase-native-client/core/async-batch-rpc-retrying-caller.h
+++ /dev/null
@@ -1,185 +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.
- *
- */
-
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/Format.h>
-#include <folly/Try.h>
-#include <folly/futures/Future.h>
-#include <folly/futures/Promise.h>
-#include <folly/io/IOBuf.h>
-#include <folly/io/async/HHWheelTimer.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-
-#include <algorithm>
-#include <chrono>
-#include <functional>
-#include <map>
-#include <memory>
-#include <mutex>
-#include <stdexcept>
-#include <string>
-#include <tuple>
-#include <type_traits>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include "connection/rpc-client.h"
-#include "core/action.h"
-#include "core/async-connection.h"
-#include "core/location-cache.h"
-#include "core/multi-response.h"
-#include "core/region-location.h"
-#include "core/region-request.h"
-#include "core/region-result.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-#include "core/result.h"
-#include "core/row.h"
-#include "core/server-request.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-#include "security/user.h"
-#include "utils/connection-util.h"
-#include "utils/sys-util.h"
-#include "utils/time-util.h"
-
-namespace hbase {
-/* Equals function for ServerName */
-struct ServerNameEquals {
-  bool operator()(const std::shared_ptr<pb::ServerName> &lhs,
-                  const std::shared_ptr<pb::ServerName> &rhs) const {
-    return (lhs->start_code() == rhs->start_code() && lhs->host_name() == rhs->host_name() &&
-            lhs->port() == rhs->port());
-  }
-};
-
-struct ServerNameHash {
-  /** hash */
-  std::size_t operator()(const std::shared_ptr<pb::ServerName> &sn) const {
-    std::size_t h = 0;
-    boost::hash_combine(h, sn->start_code());
-    boost::hash_combine(h, sn->host_name());
-    boost::hash_combine(h, sn->port());
-    return h;
-  }
-};
-
-template <typename REQ, typename RESP>
-class AsyncBatchRpcRetryingCaller {
- public:
-  using ActionsByServer =
-      std::unordered_map<std::shared_ptr<pb::ServerName>, std::shared_ptr<ServerRequest>,
-                         ServerNameHash, ServerNameEquals>;
-  using ActionsByRegion = ServerRequest::ActionsByRegion;
-
-  AsyncBatchRpcRetryingCaller(std::shared_ptr<AsyncConnection> conn,
-                              std::shared_ptr<folly::HHWheelTimer> retry_timer,
-                              std::shared_ptr<pb::TableName> table_name,
-                              const std::vector<REQ> &actions, std::chrono::nanoseconds pause_ns,
-                              int32_t max_attempts, std::chrono::nanoseconds operation_timeout_ns,
-                              std::chrono::nanoseconds rpc_timeout_ns,
-                              int32_t start_log_errors_count);
-
-  ~AsyncBatchRpcRetryingCaller();
-
-  folly::Future<std::vector<folly::Try<RESP>>> Call();
-
- private:
-  int64_t RemainingTimeNs();
-
-  void LogException(int32_t tries, std::shared_ptr<RegionRequest> region_request,
-                    const folly::exception_wrapper &ew,
-                    std::shared_ptr<pb::ServerName> server_name);
-
-  void LogException(int32_t tries,
-                    const std::vector<std::shared_ptr<RegionRequest>> &region_requests,
-                    const folly::exception_wrapper &ew,
-                    std::shared_ptr<pb::ServerName> server_name);
-
-  const std::string GetExtraContextForError(std::shared_ptr<pb::ServerName> server_name);
-
-  void AddError(const std::shared_ptr<Action> &action, const folly::exception_wrapper &ew,
-                std::shared_ptr<pb::ServerName> server_name);
-
-  void AddError(const std::vector<std::shared_ptr<Action>> &actions,
-                const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
-
-  void FailOne(const std::shared_ptr<Action> &action, int32_t tries,
-               const folly::exception_wrapper &ew, int64_t current_time, const std::string extras);
-
-  void FailAll(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries,
-               const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
-
-  void FailAll(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
-
-  void AddAction2Error(uint64_t action_index, const ThrowableWithExtraContext &twec);
-
-  void OnError(const ActionsByRegion &actions_by_region, int32_t tries,
-               const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
-
-  void TryResubmit(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
-
-  folly::Future<std::vector<folly::Try<std::shared_ptr<RegionLocation>>>> GetRegionLocations(
-      const std::vector<std::shared_ptr<Action>> &actions, int64_t locate_timeout_ns);
-
-  void GroupAndSend(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
-
-  folly::Future<std::vector<folly::Try<std::unique_ptr<Response>>>> GetMultiResponse(
-      const ActionsByServer &actions_by_server);
-
-  void Send(const ActionsByServer &actions_by_server, int32_t tries);
-
-  void OnComplete(const ActionsByRegion &actions_by_region, int32_t tries,
-                  const std::shared_ptr<pb::ServerName> server_name,
-                  const std::unique_ptr<MultiResponse> multi_results);
-
-  void OnComplete(const std::shared_ptr<Action> &action,
-                  const std::shared_ptr<RegionRequest> &region_request, int32_t tries,
-                  const std::shared_ptr<pb::ServerName> &server_name,
-                  const std::shared_ptr<RegionResult> &region_result,
-                  std::vector<std::shared_ptr<Action>> &failed_actions);
-
- private:
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<hbase::AsyncConnection> conn_;
-  std::shared_ptr<pb::TableName> table_name_;
-  std::vector<std::shared_ptr<Action>> actions_;
-  std::chrono::nanoseconds pause_ns_;
-  int32_t max_attempts_ = 0;
-  std::chrono::nanoseconds operation_timeout_ns_;
-  std::chrono::nanoseconds rpc_timeout_ns_;
-  int32_t start_log_errors_count_ = 0;
-
-  int64_t start_ns_ = TimeUtil::GetNowNanos();
-  int32_t tries_ = 1;
-  std::map<uint64_t, folly::Promise<RESP>> action2promises_;
-  std::vector<folly::Future<RESP>> action2futures_;
-  std::map<uint64_t, std::shared_ptr<std::vector<ThrowableWithExtraContext>>> action2errors_;
-
-  std::shared_ptr<AsyncRegionLocator> location_cache_ = nullptr;
-  std::shared_ptr<RpcClient> rpc_client_ = nullptr;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_pool_ = nullptr;
-
-  std::recursive_mutex multi_mutex_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-batch-rpc-retrying-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-batch-rpc-retrying-test.cc b/hbase-native-client/core/async-batch-rpc-retrying-test.cc
deleted file mode 100644
index 00cf2b8..0000000
--- a/hbase-native-client/core/async-batch-rpc-retrying-test.cc
+++ /dev/null
@@ -1,577 +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 <folly/Logging.h>
-#include <folly/Memory.h>
-#include <folly/futures/Future.h>
-#include <folly/io/async/EventBase.h>
-#include <folly/io/async/ScopedEventBaseThread.h>
-#include <gtest/gtest.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-
-#include <chrono>
-#include <functional>
-#include <string>
-
-#include "connection/rpc-client.h"
-#include "core/async-batch-rpc-retrying-caller.h"
-#include "core/async-connection.h"
-#include "core/async-rpc-retrying-caller-factory.h"
-#include "core/client.h"
-#include "core/connection-configuration.h"
-#include "core/keyvalue-codec.h"
-#include "core/region-location.h"
-#include "core/result.h"
-#include "exceptions/exception.h"
-#include "test-util/test-util.h"
-#include "utils/time-util.h"
-
-using hbase::AsyncRpcRetryingCallerFactory;
-using hbase::AsyncConnection;
-using hbase::AsyncRegionLocator;
-using hbase::ConnectionConfiguration;
-using hbase::Configuration;
-using hbase::HBaseRpcController;
-using hbase::RegionLocation;
-using hbase::RegionLocateType;
-using hbase::RpcClient;
-using hbase::RequestConverter;
-using hbase::ResponseConverter;
-using hbase::Put;
-using hbase::TimeUtil;
-using hbase::Client;
-using hbase::security::User;
-
-using std::chrono::nanoseconds;
-using std::chrono::milliseconds;
-
-using namespace hbase;
-
-using folly::exception_wrapper;
-
-class AsyncBatchRpcRetryTest : public ::testing::Test {
- public:
-  static std::unique_ptr<hbase::TestUtil> test_util;
-  static std::string tableName;
-
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util = std::make_unique<hbase::TestUtil>();
-    test_util->StartMiniCluster(2);
-    std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
-                                  "test500", "test600", "test700", "test800", "test900"};
-    tableName = "split-table1";
-    test_util->CreateTable(tableName, "d", keys);
-  }
-};
-std::unique_ptr<hbase::TestUtil> AsyncBatchRpcRetryTest::test_util = nullptr;
-std::string AsyncBatchRpcRetryTest::tableName;
-
-class AsyncRegionLocatorBase : public AsyncRegionLocator {
- public:
-  AsyncRegionLocatorBase() {}
-  explicit AsyncRegionLocatorBase(std::shared_ptr<RegionLocation> region_location)
-      : region_location_(region_location) {}
-  virtual ~AsyncRegionLocatorBase() = default;
-
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(const hbase::pb::TableName &,
-                                                                     const std::string &row,
-                                                                     const RegionLocateType,
-                                                                     const int64_t) override {
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    promise.setValue(region_locations_.at(row));
-    return promise.getFuture();
-  }
-
-  virtual void set_region_location(std::shared_ptr<RegionLocation> region_location) {
-    region_location_ = region_location;
-  }
-
-  virtual void set_region_location(
-      const std::map<std::string, std::shared_ptr<RegionLocation>> &reg_locs) {
-    for (auto reg_loc : reg_locs) {
-      region_locations_[reg_loc.first] = reg_loc.second;
-    }
-  }
-
-  void UpdateCachedLocation(const RegionLocation &rl, const folly::exception_wrapper &ew) override {
-  }
-
- protected:
-  std::shared_ptr<RegionLocation> region_location_;
-  std::map<std::string, std::shared_ptr<RegionLocation>> region_locations_;
-  std::map<std::string, uint32_t> mtries_;
-  std::map<std::string, uint32_t> mnum_fails_;
-
-  void InitRetryMaps(uint32_t num_fails) {
-    if (mtries_.size() == 0 && mnum_fails_.size() == 0) {
-      for (auto reg_loc : region_locations_) {
-        mtries_[reg_loc.first] = 0;
-        mnum_fails_[reg_loc.first] = num_fails;
-      }
-    }
-  }
-};
-
-class MockAsyncRegionLocator : public AsyncRegionLocatorBase {
- public:
-  MockAsyncRegionLocator() : AsyncRegionLocatorBase() {}
-  explicit MockAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockAsyncRegionLocator() {}
-};
-
-class MockWrongRegionAsyncRegionLocator : public AsyncRegionLocatorBase {
- private:
-  uint32_t counter_ = 0;
-  uint32_t num_fails_ = 0;
-  uint32_t tries_ = 0;
-
- public:
-  explicit MockWrongRegionAsyncRegionLocator(uint32_t num_fails)
-      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
-  explicit MockWrongRegionAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockWrongRegionAsyncRegionLocator() {}
-
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) override {
-    InitRetryMaps(num_fails_);
-    auto &tries = mtries_[row];
-    auto &num_fails = mnum_fails_[row];
-    if (++tries > num_fails) {
-      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
-    }
-
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    /* set random region name, simulating invalid region */
-    auto result = std::make_shared<RegionLocation>("whatever-region-name",
-                                                   region_locations_.at(row)->region_info(),
-                                                   region_locations_.at(row)->server_name());
-    promise.setValue(result);
-    return promise.getFuture();
-  }
-};
-
-class MockFailingAsyncRegionLocator : public AsyncRegionLocatorBase {
- private:
-  uint32_t tries_ = 0;
-  uint32_t num_fails_ = 0;
-  uint32_t counter_ = 0;
-
- public:
-  explicit MockFailingAsyncRegionLocator(uint32_t num_fails)
-      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
-  explicit MockFailingAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockFailingAsyncRegionLocator() {}
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) override {
-    InitRetryMaps(num_fails_);
-    auto &tries = mtries_[row];
-    auto &num_fails = mnum_fails_[row];
-    if (++tries > num_fails) {
-      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
-    }
-
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    promise.setException(std::runtime_error{"Failed to look up region location"});
-    return promise.getFuture();
-  }
-};
-
-class MockAsyncConnection : public AsyncConnection,
-                            public std::enable_shared_from_this<MockAsyncConnection> {
- public:
-  MockAsyncConnection(std::shared_ptr<ConnectionConfiguration> conn_conf,
-                      std::shared_ptr<folly::HHWheelTimer> retry_timer,
-                      std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                      std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                      std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor,
-                      std::shared_ptr<RpcClient> rpc_client,
-                      std::shared_ptr<AsyncRegionLocator> region_locator)
-      : conn_conf_(conn_conf),
-        retry_timer_(retry_timer),
-        cpu_executor_(cpu_executor),
-        io_executor_(io_executor),
-        retry_executor_(retry_executor),
-        rpc_client_(rpc_client),
-        region_locator_(region_locator) {}
-  ~MockAsyncConnection() {}
-  void Init() {
-    caller_factory_ =
-        std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
-  }
-
-  std::shared_ptr<Configuration> conf() override { return nullptr; }
-  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return conn_conf_; }
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
-    return caller_factory_;
-  }
-  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
-  std::shared_ptr<AsyncRegionLocator> region_locator() override { return region_locator_; }
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
-    return retry_executor_;
-  }
-
-  void Close() override {
-    retry_timer_->destroy();
-    retry_executor_->stop();
-    io_executor_->stop();
-    cpu_executor_->stop();
-  }
-  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
-    return std::make_shared<HBaseRpcController>();
-  }
-
- private:
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<ConnectionConfiguration> conn_conf_;
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
-  std::shared_ptr<RpcClient> rpc_client_;
-  std::shared_ptr<AsyncRegionLocator> region_locator_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
-};
-
-class MockRawAsyncTableImpl {
- public:
-  explicit MockRawAsyncTableImpl(std::shared_ptr<MockAsyncConnection> conn,
-                                 std::shared_ptr<hbase::pb::TableName> tn)
-      : conn_(conn), tn_(tn) {}
-  virtual ~MockRawAsyncTableImpl() = default;
-
-  /* implement this in real RawAsyncTableImpl. */
-  template <typename REQ, typename RESP>
-  folly::Future<std::vector<folly::Try<RESP>>> Batch(const std::vector<REQ> &rows) {
-    /* init request caller builder */
-    auto builder = conn_->caller_factory()->Batch<REQ, RESP>();
-
-    /* call with retry to get result */
-    auto async_caller =
-        builder->table(tn_)
-            ->actions(std::make_shared<std::vector<REQ>>(rows))
-            ->rpc_timeout(conn_->connection_conf()->read_rpc_timeout())
-            ->operation_timeout(conn_->connection_conf()->operation_timeout())
-            ->pause(conn_->connection_conf()->pause())
-            ->max_attempts(conn_->connection_conf()->max_retries())
-            ->start_log_errors_count(conn_->connection_conf()->start_log_errors_count())
-            ->Build();
-
-    return async_caller->Call().then([async_caller](auto r) { return r; });
-  }
-
- private:
-  std::shared_ptr<MockAsyncConnection> conn_;
-  std::shared_ptr<hbase::pb::TableName> tn_;
-};
-
-std::shared_ptr<MockAsyncConnection> getAsyncConnection(
-    Client &client, uint32_t operation_timeout_millis, uint32_t tries,
-    std::shared_ptr<AsyncRegionLocatorBase> region_locator) {
-  /* init region location and rpc channel */
-  auto cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
-  auto io_executor_ = client.async_connection()->io_executor();
-  auto retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
-  auto codec = std::make_shared<hbase::KeyValueCodec>();
-  auto rpc_client = std::make_shared<RpcClient>(io_executor_, cpu_executor_, codec,
-                                                AsyncBatchRpcRetryTest::test_util->conf());
-  std::shared_ptr<folly::HHWheelTimer> retry_timer =
-      folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
-
-  /* init connection configuration */
-  auto connection_conf = std::make_shared<ConnectionConfiguration>(
-      TimeUtil::SecondsToNanos(20),                       // connect_timeout
-      TimeUtil::MillisToNanos(operation_timeout_millis),  // operation_timeout
-      TimeUtil::SecondsToNanos(60),                       // rpc_timeout
-      TimeUtil::MillisToNanos(100),                       // pause
-      tries,                                              // max retries
-      1);                                                 // start log errors count
-
-  return std::make_shared<MockAsyncConnection>(connection_conf, retry_timer, cpu_executor_,
-                                               io_executor_, retry_executor_, rpc_client,
-                                               region_locator);
-}
-
-template <typename ACTION>
-std::vector<std::shared_ptr<hbase::Row>> getRows(std::vector<ACTION> actions) {
-  std::vector<std::shared_ptr<hbase::Row>> rows;
-  for (auto action : actions) {
-    std::shared_ptr<hbase::Row> srow = std::make_shared<ACTION>(action);
-    rows.push_back(srow);
-  }
-  return rows;
-}
-
-template <typename REQ, typename RESP>
-std::vector<std::shared_ptr<hbase::Result>> getResults(std::vector<REQ> &actions,
-                                                       std::vector<folly::Try<RESP>> &tresults) {
-  std::vector<std::shared_ptr<hbase::Result>> results{};
-  uint64_t num = 0;
-  for (auto tresult : tresults) {
-    if (tresult.hasValue()) {
-      results.push_back(tresult.value());
-    } else if (tresult.hasException()) {
-      folly::exception_wrapper ew = tresult.exception();
-      LOG(ERROR) << "Caught exception:- " << ew.what().toStdString() << " for "
-                 << actions[num].row();
-      throw ew;
-    }
-    ++num;
-  }
-  return results;
-}
-
-template <typename ACTION>
-std::map<std::string, std::shared_ptr<RegionLocation>> getRegionLocationsAndActions(
-    uint64_t num_rows, std::vector<ACTION> &actions, std::shared_ptr<Table> table) {
-  std::map<std::string, std::shared_ptr<RegionLocation>> region_locations;
-  for (uint64_t i = 0; i < num_rows; ++i) {
-    auto row = "test" + std::to_string(i);
-    ACTION action(row);
-    actions.push_back(action);
-    region_locations[row] = table->GetRegionLocation(row);
-  }
-  return region_locations;
-}
-
-void runMultiGets(std::shared_ptr<AsyncRegionLocatorBase> region_locator,
-                  const std::string &table_name, bool split_regions, uint32_t tries = 3,
-                  uint32_t operation_timeout_millis = 600000, uint64_t num_rows = 1000) {
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(AsyncBatchRpcRetryTest::tableName);
-
-  // Create a client
-  Client client(*AsyncBatchRpcRetryTest::test_util->conf());
-
-  // Get connection to HBase Table
-  std::shared_ptr<Table> table = client.Table(tn);
-
-  for (uint64_t i = 0; i < num_rows; i++) {
-    table->Put(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
-                                                         "value" + std::to_string(i)));
-  }
-  std::vector<hbase::Get> gets;
-  auto region_locations = getRegionLocationsAndActions<hbase::Get>(num_rows, gets, table);
-
-  /* set region locator */
-  region_locator->set_region_location(region_locations);
-
-  /* init hbase client connection */
-  auto conn = getAsyncConnection(client, operation_timeout_millis, tries, region_locator);
-  conn->Init();
-
-  /* init retry caller factory */
-  auto tableImpl =
-      std::make_shared<MockRawAsyncTableImpl>(conn, std::make_shared<hbase::pb::TableName>(tn));
-
-  std::vector<std::shared_ptr<hbase::Row>> rows = getRows<hbase::Get>(gets);
-  auto tresults = tableImpl->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(rows).get(
-      milliseconds(operation_timeout_millis));
-  ASSERT_TRUE(!tresults.empty()) << "tresults shouldn't be empty.";
-
-  auto results = getResults<hbase::Get, std::shared_ptr<Result>>(gets, tresults);
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!results.empty()) << "Results shouldn't be empty.";
-  uint32_t i = 0;
-  for (; i < num_rows; ++i) {
-    ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row()
-                                        << " must not be empty";
-    EXPECT_EQ("test" + std::to_string(i), results[i]->Row());
-    EXPECT_EQ("value" + std::to_string(i), results[i]->Value("d", std::to_string(i)).value());
-  }
-
-  table->Close();
-  client.Close();
-  conn->Close();
-}
-
-void runMultiPuts(std::shared_ptr<AsyncRegionLocatorBase> region_locator,
-                  const std::string &table_name, bool split_regions, uint32_t tries = 3,
-                  uint32_t operation_timeout_millis = 600000, uint32_t num_rows = 1000) {
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(AsyncBatchRpcRetryTest::tableName);
-
-  // Create a client
-  Client client(*AsyncBatchRpcRetryTest::test_util->conf());
-
-  // Get connection to HBase Table
-  std::shared_ptr<Table> table = client.Table(tn);
-
-  std::vector<hbase::Put> puts;
-  auto region_locations = getRegionLocationsAndActions<hbase::Put>(num_rows, puts, table);
-
-  /* set region locator */
-  region_locator->set_region_location(region_locations);
-
-  /* init hbase client connection */
-  auto conn = getAsyncConnection(client, operation_timeout_millis, tries, region_locator);
-  conn->Init();
-
-  /* init retry caller factory */
-  auto tableImpl =
-      std::make_shared<MockRawAsyncTableImpl>(conn, std::make_shared<hbase::pb::TableName>(tn));
-
-  std::vector<std::shared_ptr<hbase::Row>> rows = getRows<hbase::Put>(puts);
-  auto tresults = tableImpl->Batch<std::shared_ptr<hbase::Row>, std::shared_ptr<Result>>(rows).get(
-      milliseconds(operation_timeout_millis));
-  ASSERT_TRUE(!tresults.empty()) << "tresults shouldn't be empty.";
-
-  auto results = getResults<hbase::Put, std::shared_ptr<Result>>(puts, tresults);
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!results.empty()) << "Results shouldn't be empty.";
-
-  table->Close();
-  client.Close();
-  conn->Close();
-}
-
-// Test successful case
-TEST_F(AsyncBatchRpcRetryTest, MultiGets) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockAsyncRegionLocator>());
-  runMultiGets(region_locator, "table1", false);
-}
-
-// Tests the RPC failing 3 times, then succeeding
-TEST_F(AsyncBatchRpcRetryTest, HandleException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
-  runMultiGets(region_locator, "table2", false, 5);
-}
-
-// Tests the RPC failing 4 times, throwing an exception
-TEST_F(AsyncBatchRpcRetryTest, FailWithException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
-  EXPECT_ANY_THROW(runMultiGets(region_locator, "table3", false));
-}
-
-// Tests the region location lookup failing 3 times, then succeeding
-TEST_F(AsyncBatchRpcRetryTest, HandleExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(3));
-  runMultiGets(region_locator, "table4", false);
-}
-
-// Tests the region location lookup failing 5 times, throwing an exception
-TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(4));
-  EXPECT_ANY_THROW(runMultiGets(region_locator, "table5", false, 3));
-}
-
-// Tests hitting operation timeout, thus not retrying anymore
-TEST_F(AsyncBatchRpcRetryTest, FailWithOperationTimeout) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(6));
-  EXPECT_ANY_THROW(runMultiGets(region_locator, "table6", false, 5, 100, 1000));
-}
-
-//////////////////////
-// Test successful case
-TEST_F(AsyncBatchRpcRetryTest, MultiPuts) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockAsyncRegionLocator>());
-  runMultiPuts(region_locator, "table1", false);
-}
-
-// Tests the RPC failing 3 times, then succeeding
-TEST_F(AsyncBatchRpcRetryTest, PutsHandleException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
-  runMultiPuts(region_locator, "table2", false, 5);
-}
-
-// Tests the RPC failing 4 times, throwing an exception
-TEST_F(AsyncBatchRpcRetryTest, PutsFailWithException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
-  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table3", false));
-}
-
-// Tests the region location lookup failing 3 times, then succeeding
-TEST_F(AsyncBatchRpcRetryTest, PutsHandleExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(3));
-  runMultiPuts(region_locator, "table4", false);
-}
-
-// Tests the region location lookup failing 5 times, throwing an exception
-TEST_F(AsyncBatchRpcRetryTest, PutsFailWithExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(4));
-  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table5", false, 3));
-}
-
-// Tests hitting operation timeout, thus not retrying anymore
-TEST_F(AsyncBatchRpcRetryTest, PutsFailWithOperationTimeout) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(6));
-  EXPECT_ANY_THROW(runMultiPuts(region_locator, "table6", false, 5, 100, 1000));
-}
-
- // Test successful case
- TEST_F(AsyncBatchRpcRetryTest, MultiGetsSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockAsyncRegionLocator>());
- runMultiGets(region_locator, "table7", true);
- }
-
- // Tests the RPC failing 3 times, then succeeding
- TEST_F(AsyncBatchRpcRetryTest, HandleExceptionSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
- runMultiGets(region_locator, "table8", true, 5);
- }
-
- // Tests the RPC failing 4 times, throwing an exception
- TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockWrongRegionAsyncRegionLocator>(4));
- EXPECT_ANY_THROW(runMultiGets(region_locator, "table9", true));
- }
-
- // Tests the region location lookup failing 3 times, then succeeding
- TEST_F(AsyncBatchRpcRetryTest, HandleExceptionFromRegionLocationLookupSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockFailingAsyncRegionLocator>(3));
- runMultiGets(region_locator, "table10", true);
- }
-
- // Tests the region location lookup failing 5 times, throwing an exception
- TEST_F(AsyncBatchRpcRetryTest, FailWithExceptionFromRegionLocationLookupSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockFailingAsyncRegionLocator>(4));
- EXPECT_ANY_THROW(runMultiGets(region_locator, "table11", true, 3));
- }
-
- // Tests hitting operation timeout, thus not retrying anymore
- TEST_F(AsyncBatchRpcRetryTest, FailWithOperationTimeoutSplitRegions) {
- std::shared_ptr<AsyncRegionLocatorBase> region_locator(
- std::make_shared<MockFailingAsyncRegionLocator>(6));
- EXPECT_ANY_THROW(runMultiGets(region_locator, "table12", true, 5, 100, 1000));
- }

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-client-scanner.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-client-scanner.cc b/hbase-native-client/core/async-client-scanner.cc
deleted file mode 100644
index 720ab25..0000000
--- a/hbase-native-client/core/async-client-scanner.cc
+++ /dev/null
@@ -1,142 +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/async-client-scanner.h"
-
-#include <algorithm>
-#include <iterator>
-#include <limits>
-#include <stdexcept>
-
-namespace hbase {
-
-AsyncClientScanner::AsyncClientScanner(
-    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<Scan> scan,
-    std::shared_ptr<pb::TableName> table_name, std::shared_ptr<RawScanResultConsumer> consumer,
-    nanoseconds pause, uint32_t max_retries, nanoseconds scan_timeout_nanos,
-    nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
-    : conn_(conn),
-      scan_(scan),
-      table_name_(table_name),
-      consumer_(consumer),
-      pause_(pause),
-      max_retries_(max_retries),
-      scan_timeout_nanos_(scan_timeout_nanos),
-      rpc_timeout_nanos_(rpc_timeout_nanos),
-      start_log_errors_count_(start_log_errors_count) {
-  results_cache_ = std::make_shared<ScanResultCache>();
-  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
-}
-
-void AsyncClientScanner::Start() { OpenScanner(); }
-
-folly::Future<std::shared_ptr<OpenScannerResponse>> AsyncClientScanner::CallOpenScanner(
-    std::shared_ptr<hbase::RpcClient> rpc_client,
-    std::shared_ptr<hbase::HBaseRpcController> controller,
-    std::shared_ptr<hbase::RegionLocation> loc) {
-  open_scanner_tries_++;
-
-  auto preq = RequestConverter::ToScanRequest(*scan_, loc->region_name(), scan_->Caching(), false);
-
-  auto self(shared_from_this());
-  VLOG(5) << "Calling RPC Client to open the scanner for region:" << loc->DebugString();
-  return rpc_client
-      ->AsyncCall(loc->server_name().host_name(), loc->server_name().port(), std::move(preq),
-                  security::User::defaultUser(), "ClientService")
-      .then([self, loc, controller, rpc_client](const std::unique_ptr<Response>& presp) {
-        VLOG(5) << "Scan Response:" << presp->DebugString();
-        return std::make_shared<OpenScannerResponse>(rpc_client, presp, loc, controller);
-      });
-}
-
-void AsyncClientScanner::OpenScanner() {
-  auto self(shared_from_this());
-  open_scanner_tries_ = 1;
-
-  auto caller = conn_->caller_factory()
-                    ->Single<std::shared_ptr<OpenScannerResponse>>()
-                    ->table(table_name_)
-                    ->row(scan_->StartRow())
-                    ->locate_type(GetLocateType(*scan_))
-                    ->rpc_timeout(rpc_timeout_nanos_)
-                    ->operation_timeout(scan_timeout_nanos_)
-                    ->pause(pause_)
-                    ->max_retries(max_retries_)
-                    ->start_log_errors_count(start_log_errors_count_)
-                    ->action([&](std::shared_ptr<hbase::HBaseRpcController> controller,
-                                 std::shared_ptr<hbase::RegionLocation> loc,
-                                 std::shared_ptr<hbase::RpcClient> rpc_client)
-                                 -> folly::Future<std::shared_ptr<OpenScannerResponse>> {
-                                   return CallOpenScanner(rpc_client, controller, loc);
-                                 })
-                    ->Build();
-
-  caller->Call()
-      .then([this, self](std::shared_ptr<OpenScannerResponse> resp) {
-        VLOG(3) << "Opened scanner with id:" << resp->scan_resp_->scanner_id()
-                << ", region:" << resp->region_location_->DebugString() << ", starting scan";
-        StartScan(resp);
-      })
-      .onError([this, self](const folly::exception_wrapper& e) {
-        VLOG(3) << "Open scan request received error:" << e.what();
-        consumer_->OnError(e);
-      })
-      .then([caller, self](const auto r) { return r; });
-}
-
-void AsyncClientScanner::StartScan(std::shared_ptr<OpenScannerResponse> resp) {
-  auto self(shared_from_this());
-  auto caller = conn_->caller_factory()
-                    ->Scan()
-                    ->scanner_id(resp->scan_resp_->scanner_id())
-                    ->region_location(resp->region_location_)
-                    ->scanner_lease_timeout(TimeUtil::MillisToNanos(resp->scan_resp_->ttl()))
-                    ->scan(scan_)
-                    ->rpc_client(resp->rpc_client_)
-                    ->consumer(consumer_)
-                    ->results_cache(results_cache_)
-                    ->rpc_timeout(rpc_timeout_nanos_)
-                    ->scan_timeout(scan_timeout_nanos_)
-                    ->pause(pause_)
-                    ->max_retries(max_retries_)
-                    ->start_log_errors_count(start_log_errors_count_)
-                    ->Build();
-
-  caller->Start(resp->controller_, resp->scan_resp_, resp->cell_scanner_)
-      .then([caller, self](const bool has_more) {
-        if (has_more) {
-          // open the next scanner on the next region.
-          self->OpenScanner();
-        } else {
-          self->consumer_->OnComplete();
-        }
-      })
-      .onError([caller, self](const folly::exception_wrapper& e) { self->consumer_->OnError(e); })
-      .then([caller, self](const auto r) { return r; });
-}
-
-RegionLocateType AsyncClientScanner::GetLocateType(const Scan& scan) {
-  // TODO: In C++, there is no Scan::IncludeStartRow() and Scan::IncludeStopRow().
-  // When added, this method should be modified to return other RegionLocateTypes
-  // (see ConnectionUtils.java #getLocateType())
-  // TODO: When reversed scans are implemented, return other RegionLocateTypes
-  return RegionLocateType::kCurrent;
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-client-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-client-scanner.h b/hbase-native-client/core/async-client-scanner.h
deleted file mode 100644
index 8663468..0000000
--- a/hbase-native-client/core/async-client-scanner.h
+++ /dev/null
@@ -1,119 +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.
- *
- */
-#pragma once
-
-#include <folly/Format.h>
-#include <folly/Logging.h>
-#include <folly/futures/Future.h>
-#include <folly/io/async/EventBase.h>
-#include <folly/io/async/HHWheelTimer.h>
-
-#include <algorithm>
-#include <chrono>
-#include <functional>
-#include <memory>
-#include <string>
-#include <type_traits>
-#include <utility>
-#include <vector>
-
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/async-rpc-retrying-caller-factory.h"
-#include "core/async-rpc-retrying-caller.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/raw-scan-result-consumer.h"
-#include "core/region-location.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-#include "core/result.h"
-#include "core/scan-result-cache.h"
-#include "core/scan.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-#include "utils/connection-util.h"
-#include "utils/sys-util.h"
-#include "utils/time-util.h"
-
-using std::chrono::nanoseconds;
-using std::chrono::milliseconds;
-
-namespace hbase {
-class OpenScannerResponse {
- public:
-  OpenScannerResponse(std::shared_ptr<hbase::RpcClient> rpc_client,
-                      const std::unique_ptr<Response>& resp,
-                      std::shared_ptr<RegionLocation> region_location,
-                      std::shared_ptr<hbase::HBaseRpcController> controller)
-      : rpc_client_(rpc_client), region_location_(region_location), controller_(controller) {
-    scan_resp_ = std::static_pointer_cast<pb::ScanResponse>(resp->resp_msg());
-    cell_scanner_ = resp->cell_scanner();
-  }
-  std::shared_ptr<hbase::RpcClient> rpc_client_;
-  std::shared_ptr<pb::ScanResponse> scan_resp_;
-  std::shared_ptr<RegionLocation> region_location_;
-  std::shared_ptr<hbase::HBaseRpcController> controller_;
-  std::shared_ptr<CellScanner> cell_scanner_;
-};
-
-class AsyncClientScanner : public std::enable_shared_from_this<AsyncClientScanner> {
- public:
-  template <typename... T>
-  static std::shared_ptr<AsyncClientScanner> Create(T&&... all) {
-    return std::shared_ptr<AsyncClientScanner>(new AsyncClientScanner(std::forward<T>(all)...));
-  }
-
-  void Start();
-
- private:
-  // methods
-  AsyncClientScanner(std::shared_ptr<AsyncConnection> conn, std::shared_ptr<Scan> scan,
-                     std::shared_ptr<pb::TableName> table_name,
-                     std::shared_ptr<RawScanResultConsumer> consumer, nanoseconds pause,
-                     uint32_t max_retries, nanoseconds scan_timeout_nanos,
-                     nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
-
-  folly::Future<std::shared_ptr<OpenScannerResponse>> CallOpenScanner(
-      std::shared_ptr<hbase::RpcClient> rpc_client,
-      std::shared_ptr<hbase::HBaseRpcController> controller,
-      std::shared_ptr<hbase::RegionLocation> loc);
-
-  void OpenScanner();
-
-  void StartScan(std::shared_ptr<OpenScannerResponse> resp);
-
-  RegionLocateType GetLocateType(const Scan& scan);
-
- private:
-  // data
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<Scan> scan_;
-  std::shared_ptr<pb::TableName> table_name_;
-  std::shared_ptr<ScanResultCache> results_cache_;
-  std::shared_ptr<RawScanResultConsumer> consumer_;
-  nanoseconds pause_;
-  uint32_t max_retries_;
-  nanoseconds scan_timeout_nanos_;
-  nanoseconds rpc_timeout_nanos_;
-  uint32_t start_log_errors_count_;
-  uint32_t max_attempts_;
-  uint32_t open_scanner_tries_ = 0;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-connection.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-connection.cc b/hbase-native-client/core/async-connection.cc
deleted file mode 100644
index 850fb8f..0000000
--- a/hbase-native-client/core/async-connection.cc
+++ /dev/null
@@ -1,70 +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/async-connection.h"
-#include "core/async-rpc-retrying-caller-factory.h"
-
-namespace hbase {
-
-void AsyncConnectionImpl::Init() {
-  connection_conf_ = std::make_shared<hbase::ConnectionConfiguration>(*conf_);
-  // start thread pools
-  auto io_threads = conf_->GetInt(kClientIoThreadPoolSize, sysconf(_SC_NPROCESSORS_ONLN));
-  auto cpu_threads = conf_->GetInt(kClientCpuThreadPoolSize, 2 * sysconf(_SC_NPROCESSORS_ONLN));
-  cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(cpu_threads);
-  io_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(io_threads);
-  /*
-   * We need a retry_executor for a thread pool of size 1 due to a possible bug in wangle/folly.
-   * Otherwise, Assertion 'isInEventBaseThread()' always fails. See the comments
-   * in async-rpc-retrying-caller.cc.
-   */
-  retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
-  retry_timer_ = folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
-
-  std::shared_ptr<Codec> codec = nullptr;
-  if (conf_->Get(kRpcCodec, hbase::KeyValueCodec::kJavaClassName) ==
-      std::string(KeyValueCodec::kJavaClassName)) {
-    codec = std::make_shared<hbase::KeyValueCodec>();
-  } else {
-    LOG(WARNING) << "Not using RPC Cell Codec";
-  }
-  rpc_client_ = std::make_shared<hbase::RpcClient>(io_executor_, cpu_executor_, codec, conf_,
-                                                   connection_conf_->connect_timeout());
-  location_cache_ = std::make_shared<hbase::LocationCache>(conf_, io_executor_, cpu_executor_,
-                                                           rpc_client_->connection_pool());
-  caller_factory_ =
-      std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
-}
-
-// We can't have the threads continue running after everything is done
-// that leads to an error.
-AsyncConnectionImpl::~AsyncConnectionImpl() { Close(); }
-
-void AsyncConnectionImpl::Close() {
-  if (is_closed_) return;
-
-  cpu_executor_->stop();
-  io_executor_->stop();
-  retry_executor_->stop();
-  retry_timer_->destroy();
-  if (rpc_client_.get()) rpc_client_->Close();
-  is_closed_ = true;
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-connection.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-connection.h b/hbase-native-client/core/async-connection.h
deleted file mode 100644
index 7b260a5..0000000
--- a/hbase-native-client/core/async-connection.h
+++ /dev/null
@@ -1,121 +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.
- *
- */
-
-#pragma once
-
-#include <folly/futures/Future.h>
-#include <folly/io/IOBuf.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-
-#include <memory>
-#include <string>
-#include <utility>
-
-#include "connection/rpc-client.h"
-#include "core/async-region-locator.h"
-#include "core/configuration.h"
-#include "core/connection-configuration.h"
-#include "core/hbase-configuration-loader.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/keyvalue-codec.h"
-#include "core/location-cache.h"
-#include "if/Cell.pb.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-
-class AsyncRpcRetryingCallerFactory;
-
-class AsyncConnection {
- public:
-  AsyncConnection() {}
-  virtual ~AsyncConnection() {}
-  virtual std::shared_ptr<Configuration> conf() = 0;
-  virtual std::shared_ptr<ConnectionConfiguration> connection_conf() = 0;
-  virtual std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() = 0;
-  virtual std::shared_ptr<RpcClient> rpc_client() = 0;
-  virtual std::shared_ptr<AsyncRegionLocator> region_locator() = 0;
-  virtual std::shared_ptr<HBaseRpcController> CreateRpcController() = 0;
-  virtual std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() = 0;
-  virtual std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() = 0;
-  virtual std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() = 0;
-  virtual void Close() = 0;
-};
-
-class AsyncConnectionImpl : public AsyncConnection,
-                            public std::enable_shared_from_this<AsyncConnectionImpl> {
- public:
-  virtual ~AsyncConnectionImpl();
-
-  // See https://mortoray.com/2013/08/02/safely-using-enable_shared_from_this/
-  template <typename... T>
-  static std::shared_ptr<AsyncConnectionImpl> Create(T&&... all) {
-    auto conn =
-        std::shared_ptr<AsyncConnectionImpl>(new AsyncConnectionImpl(std::forward<T>(all)...));
-    conn->Init();
-    return conn;
-  }
-
-  std::shared_ptr<Configuration> conf() override { return conf_; }
-  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return connection_conf_; }
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
-    return caller_factory_;
-  }
-  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
-  std::shared_ptr<LocationCache> location_cache() { return location_cache_; }
-  std::shared_ptr<AsyncRegionLocator> region_locator() override { return location_cache_; }
-  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
-    return std::make_shared<HBaseRpcController>();
-  }
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
-    return retry_executor_;
-  }
-
-  void Close() override;
-
- protected:
-  AsyncConnectionImpl() {}
-
- private:
-  /** Parameter name for HBase client IO thread pool size. Defaults to num cpus */
-  static constexpr const char* kClientIoThreadPoolSize = "hbase.client.io.thread.pool.size";
-  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
-  static constexpr const char* kClientCpuThreadPoolSize = "hbase.client.cpu.thread.pool.size";
-  /** The RPC codec to encode cells. For now it is KeyValueCodec */
-  static constexpr const char* kRpcCodec = "hbase.client.rpc.codec";
-
-  std::shared_ptr<Configuration> conf_;
-  std::shared_ptr<ConnectionConfiguration> connection_conf_;
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
-  std::shared_ptr<LocationCache> location_cache_;
-  std::shared_ptr<RpcClient> rpc_client_;
-  bool is_closed_ = false;
-
- private:
-  explicit AsyncConnectionImpl(std::shared_ptr<Configuration> conf) : conf_(conf) {}
-  void Init();
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-region-locator.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-region-locator.h b/hbase-native-client/core/async-region-locator.h
deleted file mode 100644
index f75cb7e..0000000
--- a/hbase-native-client/core/async-region-locator.h
+++ /dev/null
@@ -1,65 +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.
- *
- */
-
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/futures/Future.h>
-#include <memory>
-#include <string>
-
-#include "core/region-location.h"
-#include "if/Client.pb.h"
-#include "serde/region-info.h"
-#include "serde/server-name.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-
-class AsyncRegionLocator {
- public:
-  AsyncRegionLocator() {}
-  virtual ~AsyncRegionLocator() = default;
-
-  /**
-   * The only method clients should use for meta lookups. If corresponding
-   * location is cached, it's returned from the cache, otherwise lookup
-   * in meta table is done, location is cached and then returned.
-   * It's expected that tiny fraction of invocations incurs meta scan.
-   * This method is to look up non-meta regions; use LocateMeta() to get the
-   * location of hbase:meta region.
-   *
-   * @param tn Table name of the table to look up. This object must live until
-   * after the future is returned
-   *
-   * @param row of the table to look up. This object must live until after the
-   * future is returned
-   */
-  virtual folly::Future<std::shared_ptr<RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) = 0;
-  /**
-   * Update cached region location, possibly using the information from exception.
-   */
-  virtual void UpdateCachedLocation(const RegionLocation &loc,
-                                    const folly::exception_wrapper &error) = 0;
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-rpc-retrying-caller-factory.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-rpc-retrying-caller-factory.cc b/hbase-native-client/core/async-rpc-retrying-caller-factory.cc
deleted file mode 100644
index 0ac9cac..0000000
--- a/hbase-native-client/core/async-rpc-retrying-caller-factory.cc
+++ /dev/null
@@ -1,22 +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/async-rpc-retrying-caller-factory.h"
-
-namespace hbase {}  // namespace hbase


[17/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.cc b/hbase-native-client/core/result.cc
deleted file mode 100644
index 44b4c86..0000000
--- a/hbase-native-client/core/result.cc
+++ /dev/null
@@ -1,146 +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/result.h"
-
-namespace hbase {
-
-Result::~Result() {}
-
-Result::Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists, bool stale,
-               bool partial)
-    : exists_(exists), stale_(stale), partial_(partial), cells_(cells) {
-  row_ = (cells_.size() == 0 ? "" : cells_[0]->Row());
-}
-
-Result::Result(const Result &result) {
-  exists_ = result.exists_;
-  stale_ = result.stale_;
-  partial_ = result.partial_;
-  row_ = result.row_;
-  if (!result.cells_.empty()) {
-    for (const auto &cell : result.cells_) {
-      cells_.push_back(cell);
-    }
-  }
-}
-
-const std::vector<std::shared_ptr<Cell> > &Result::Cells() const { return cells_; }
-
-std::vector<std::shared_ptr<Cell> > Result::ColumnCells(const std::string &family,
-                                                        const std::string &qualifier) const {
-  std::vector<std::shared_ptr<Cell> > column_cells;
-  // TODO implement a BinarySearch here ?
-  for (const auto &cell : cells_) {
-    if (cell->Family() == family && cell->Qualifier() == qualifier) {
-      column_cells.push_back(cell);
-    }
-  }
-  return column_cells;
-}
-
-const std::shared_ptr<Cell> Result::ColumnLatestCell(const std::string &family,
-                                                     const std::string &qualifier) const {
-  // TODO implement a BinarySearch here ?
-  for (const auto &cell : cells_) {
-    // We find the latest(first) occurrence of the Cell for a given column and
-    // qualifier and break
-    if (cell->Family() == family && cell->Qualifier() == qualifier) {
-      return cell;
-    }
-  }
-  return nullptr;
-}
-
-optional<std::string> Result::Value(const std::string &family, const std::string &qualifier) const {
-  std::shared_ptr<Cell> latest_cell(ColumnLatestCell(family, qualifier));
-  if (latest_cell.get()) {
-    return optional<std::string>(latest_cell->Value());
-  }
-  return optional<std::string>();
-}
-
-bool Result::IsEmpty() const { return cells_.empty(); }
-
-const std::string &Result::Row() const { return row_; }
-
-int Result::Size() const { return cells_.size(); }
-
-ResultMap Result::Map() const {
-  ResultMap result_map;
-  for (const auto &cell : cells_) {
-    result_map[cell->Family()][cell->Qualifier()][cell->Timestamp()] = cell->Value();
-  }
-  return result_map;
-}
-
-std::map<std::string, std::string> Result::FamilyMap(const std::string &family) const {
-  std::map<std::string, std::string> family_map;
-  if (!IsEmpty()) {
-    auto result_map = Map();
-    auto itr = result_map.find(family);
-    if (itr == result_map.end()) {
-      return family_map;
-    }
-
-    for (auto qitr = itr->second.begin(); qitr != itr->second.end(); ++qitr) {
-      for (auto vitr = qitr->second.begin(); vitr != qitr->second.end(); ++vitr) {
-        // We break after inserting the first value. Result.java takes only
-        // the first value
-        family_map[qitr->first] = vitr->second;
-        break;
-      }
-    }
-  }
-
-  return family_map;
-}
-
-std::string Result::DebugString() const {
-  std::string ret{"keyvalues="};
-  if (IsEmpty()) {
-    ret += "NONE";
-    return ret;
-  }
-  ret += "{";
-  bool is_first = true;
-  for (const auto &cell : cells_) {
-    if (is_first) {
-      is_first = false;
-    } else {
-      ret += ", ";
-    }
-    ret += cell->DebugString();
-  }
-  ret += "}";
-
-  return ret;
-}
-
-size_t Result::EstimatedSize() const {
-  size_t s = sizeof(Result);
-  s += row_.capacity();
-  for (const auto c : cells_) {
-    s += sizeof(std::shared_ptr<Cell>);
-    s + c->EstimatedSize();
-  }
-  return s;
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.h b/hbase-native-client/core/result.h
deleted file mode 100644
index f18071b..0000000
--- a/hbase-native-client/core/result.h
+++ /dev/null
@@ -1,136 +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.
- *
- */
-
-#pragma once
-
-#include <functional>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "core/cell.h"
-#include "utils/optional.h"
-
-namespace hbase {
-
-/**
- * @brief Map of families to all versions of its qualifiers and values
- * We need to have a reverse ordered map, when storing TS -> value, so that the
- * most recent value is stored first
- */
-using ResultMap =
-    std::map<std::string,
-             std::map<std::string, std::map<int64_t, std::string, std::greater<int64_t> > > >;
-
-class Result {
- public:
-  /**
-   * Constructors
-   */
-  Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists, bool stale, bool partial);
-  Result(const Result &result);
-  ~Result();
-
-  /**
-   * @brief Return the vector of Cells backing this Result instance. This vector
-   * will be ordered in the same manner
-   * as the one which was passed while creation of the Result instance.
-   */
-  const std::vector<std::shared_ptr<Cell> > &Cells() const;
-
-  /**
-   * @brief  Return a vector of Cells for the family and qualifier or empty list
-   * if the column
-   * did not exist in the result.
-   * @param family - column family
-   * @param qualifier - column qualifier
-   */
-  std::vector<std::shared_ptr<Cell> > ColumnCells(const std::string &family,
-                                                  const std::string &qualifier) const;
-
-  /**
-   * @brief Returns the Cell for the most recent timestamp for a given family
-   * and qualifier.
-   * Returns map of qualifiers to values, only includes latest values
-   * @param family - column family.
-   * @param qualifier - column qualifier
-   */
-  const std::shared_ptr<Cell> ColumnLatestCell(const std::string &family,
-                                               const std::string &qualifier) const;
-
-  /**
-   * @brief Get the latest version of the specified family and qualifier.
-   * @param family - column family
-   * @param qualifier - column qualifier
-   */
-  optional<std::string> Value(const std::string &family, const std::string &qualifier) const;
-
-  /**
-   * @brief Returns if the underlying Cell vector is empty or not
-   */
-  bool IsEmpty() const;
-
-  /**
-   * @brief Retrieves the row key that corresponds to the row from which this
-   * Result was created.
-   */
-  const std::string &Row() const;
-
-  /**
-   * @brief Returns the size of the underlying Cell vector
-   */
-  int Size() const;
-
-  /**
-   * @brief Map of families to all versions of its qualifiers and values.
-   * Returns a three level Map of the form:
-   * Map<family,Map<qualifier,Map<timestamp,value>>>>
-   * All other map returning methods make use of this map internally
-   * The Map is created when the Result instance is created
-   */
-  ResultMap Map() const;
-
-  /**
-   * @brief Map of qualifiers to values.
-   * Returns a Map of the form: Map<qualifier,value>
-   * @param family - column family to get
-   */
-  std::map<std::string, std::string> FamilyMap(const std::string &family) const;
-
-  std::string DebugString() const;
-
-  bool Exists() const { return exists_; }
-
-  bool Stale() const { return stale_; }
-
-  bool Partial() const { return partial_; }
-
-  /** Returns estimated size of the Result object including deep heap space usage
-   * of its Cells and data. Notice that this is a very rough estimate. */
-  size_t EstimatedSize() const;
-
- private:
-  bool exists_ = false;
-  bool stale_ = false;
-  bool partial_ = false;
-  std::string row_ = "";
-  std::vector<std::shared_ptr<Cell> > cells_;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/row.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/row.h b/hbase-native-client/core/row.h
deleted file mode 100644
index 2c7bdd1..0000000
--- a/hbase-native-client/core/row.h
+++ /dev/null
@@ -1,62 +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 <limits>
-#include <stdexcept>
-#include <string>
-
-#pragma once
-
-namespace hbase {
-
-class Row {
- public:
-  Row() {}
-  explicit Row(const std::string &row) : row_(row) { CheckRow(row_); }
-
-  /**
-   * @brief Returns the row for the Row interface.
-   */
-  const std::string &row() const { return row_; }
-  virtual ~Row() {}
-
- private:
-  /**
-   * @brief Checks if the row for this Get operation is proper or not
-   * @param row Row to check
-   * @throws std::runtime_error if row is empty or greater than
-   * MAX_ROW_LENGTH(i.e. std::numeric_limits<short>::max())
-   */
-  void CheckRow(const std::string &row) {
-    const int16_t kMaxRowLength = std::numeric_limits<int16_t>::max();
-    size_t row_length = row.size();
-    if (0 == row_length) {
-      throw std::runtime_error("Row length can't be 0");
-    }
-    if (row_length > kMaxRowLength) {
-      throw std::runtime_error("Length of " + row + " is greater than max row size: " +
-                               std::to_string(kMaxRowLength));
-    }
-  }
-
- protected:
-  std::string row_ = "";
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan-result-cache-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan-result-cache-test.cc b/hbase-native-client/core/scan-result-cache-test.cc
deleted file mode 100644
index 0bf83ce..0000000
--- a/hbase-native-client/core/scan-result-cache-test.cc
+++ /dev/null
@@ -1,177 +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 <folly/Conv.h>
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include <vector>
-
-#include "core/cell.h"
-#include "core/result.h"
-#include "core/scan-result-cache.h"
-
-using hbase::ScanResultCache;
-using hbase::Result;
-using hbase::Cell;
-using hbase::CellType;
-
-using ResultVector = std::vector<std::shared_ptr<Result>>;
-
-std::shared_ptr<Cell> CreateCell(const int32_t &key, const std::string &family,
-                                 const std::string &column) {
-  auto row = folly::to<std::string>(key);
-  return std::make_shared<Cell>(row, family, column, std::numeric_limits<int64_t>::max(), row,
-                                CellType::PUT);
-}
-
-std::shared_ptr<Result> CreateResult(std::shared_ptr<Cell> cell, bool partial) {
-  return std::make_shared<Result>(std::vector<std::shared_ptr<Cell>>{cell}, false, false, partial);
-}
-
-TEST(ScanResultCacheTest, NoPartial) {
-  ScanResultCache cache;
-  ASSERT_EQ(ResultVector{}, cache.AddAndGet(ResultVector{}, false));
-  ASSERT_EQ(ResultVector{}, cache.AddAndGet(ResultVector{}, true));
-  int32_t count = 10;
-  ResultVector results{};
-  for (int32_t i = 0; i < count; i++) {
-    results.push_back(CreateResult(CreateCell(i, "cf", "cq1"), false));
-  }
-  ASSERT_EQ(results, cache.AddAndGet(results, false));
-}
-
-TEST(ScanResultCacheTest, Combine1) {
-  ScanResultCache cache;
-  auto prev_result = CreateResult(CreateCell(0, "cf", "cq1"), true);
-  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
-  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
-  auto result3 = CreateResult(CreateCell(1, "cf", "cq3"), true);
-  auto results = cache.AddAndGet(ResultVector{prev_result, result1}, false);
-  ASSERT_EQ(1L, results.size());
-  ASSERT_EQ(prev_result, results[0]);
-
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result3}, false).size());
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{}, true).size());
-
-  results = cache.AddAndGet(ResultVector{}, false);
-  ASSERT_EQ(1, results.size());
-  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(3, results[0]->Cells().size());
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq3")));
-}
-
-TEST(ScanResultCacheTest, Combine2) {
-  ScanResultCache cache;
-  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
-  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
-  auto result3 = CreateResult(CreateCell(1, "cf", "cq3"), true);
-
-  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), true);
-  auto next_to_next_result1 = CreateResult(CreateCell(3, "cf", "cq2"), false);
-
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result3}, false).size());
-
-  auto results = cache.AddAndGet(ResultVector{next_result1}, false);
-  ASSERT_EQ(1, results.size());
-  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(3, results[0]->Cells().size());
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq3")));
-
-  results = cache.AddAndGet(ResultVector{next_to_next_result1}, false);
-  ASSERT_EQ(2, results.size());
-  ASSERT_EQ(2, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(1, results[0]->Cells().size());
-  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(3, folly::to<int32_t>(results[1]->Row()));
-  ASSERT_EQ(1, results[1]->Cells().size());
-  ASSERT_EQ(3, folly::to<int32_t>(*results[1]->Value("cf", "cq2")));
-}
-
-TEST(ScanResultCacheTest, Combine3) {
-  ScanResultCache cache;
-  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
-  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), true);
-  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), false);
-  auto next_to_next_result1 = CreateResult(CreateCell(3, "cf", "cq1"), true);
-
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result2}, false).size());
-
-  auto results = cache.AddAndGet(ResultVector{next_result1, next_to_next_result1}, false);
-
-  ASSERT_EQ(2, results.size());
-  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(2, results[0]->Cells().size());
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
-  ASSERT_EQ(2, folly::to<int32_t>(results[1]->Row()));
-  ASSERT_EQ(1, results[1]->Cells().size());
-  ASSERT_EQ(2, folly::to<int32_t>(*results[1]->Value("cf", "cq1")));
-
-  results = cache.AddAndGet(ResultVector{}, false);
-
-  ASSERT_EQ(1, results.size());
-  ASSERT_EQ(3, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(1, results[0]->Cells().size());
-  ASSERT_EQ(3, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-}
-
-TEST(ScanResultCacheTest, Combine4) {
-  ScanResultCache cache;
-  auto result1 = CreateResult(CreateCell(1, "cf", "cq1"), true);
-  auto result2 = CreateResult(CreateCell(1, "cf", "cq2"), false);
-  auto next_result1 = CreateResult(CreateCell(2, "cf", "cq1"), true);
-  auto next_result2 = CreateResult(CreateCell(2, "cf", "cq2"), false);
-
-  ASSERT_EQ(0, cache.AddAndGet(ResultVector{result1}, false).size());
-
-  auto results = cache.AddAndGet(ResultVector{result2, next_result1}, false);
-
-  ASSERT_EQ(1, results.size());
-  ASSERT_EQ(1, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(2, results[0]->Cells().size());
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(1, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
-
-  results = cache.AddAndGet(ResultVector{next_result2}, false);
-
-  ASSERT_EQ(1, results.size());
-  ASSERT_EQ(2, folly::to<int32_t>(results[0]->Row()));
-  ASSERT_EQ(2, results[0]->Cells().size());
-  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq1")));
-  ASSERT_EQ(2, folly::to<int32_t>(*results[0]->Value("cf", "cq2")));
-}
-
-TEST(ScanResultCacheTest, SizeOf) {
-  std::string e{""};
-  std::string f{"f"};
-  std::string foo{"foo"};
-
-  LOG(INFO) << sizeof(e) << " " << e.capacity();
-  LOG(INFO) << sizeof(f) << " " << f.capacity();
-  LOG(INFO) << sizeof(foo) << " " << foo.capacity();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan-result-cache.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan-result-cache.cc b/hbase-native-client/core/scan-result-cache.cc
deleted file mode 100644
index 62a51e0..0000000
--- a/hbase-native-client/core/scan-result-cache.cc
+++ /dev/null
@@ -1,160 +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/scan-result-cache.h"
-#include <algorithm>
-#include <iterator>
-#include <limits>
-#include <stdexcept>
-
-namespace hbase {
-/**
- * Add the given results to cache and get valid results back.
- * @param results the results of a scan next. Must not be null.
- * @param is_hearthbeat indicate whether the results is gotten from a heartbeat response.
- * @return valid results, never null.
- */
-std::vector<std::shared_ptr<Result>> ScanResultCache::AddAndGet(
-    const std::vector<std::shared_ptr<Result>> &results, bool is_hearthbeat) {
-  // If no results were returned it indicates that either we have the all the partial results
-  // necessary to construct the complete result or the server had to send a heartbeat message
-  // to the client to keep the client-server connection alive
-  if (results.empty()) {
-    // If this response was an empty heartbeat message, then we have not exhausted the region
-    // and thus there may be more partials server side that still need to be added to the partial
-    // list before we form the complete Result
-    if (!partial_results_.empty() && !is_hearthbeat) {
-      return UpdateNumberOfCompleteResultsAndReturn(
-          std::vector<std::shared_ptr<Result>>{Combine()});
-    }
-    return std::vector<std::shared_ptr<Result>>{};
-  }
-  // In every RPC response there should be at most a single partial result. Furthermore, if
-  // there is a partial result, it is guaranteed to be in the last position of the array.
-  auto last = results[results.size() - 1];
-  if (last->Partial()) {
-    if (partial_results_.empty()) {
-      partial_results_.push_back(last);
-      std::vector<std::shared_ptr<Result>> new_results;
-      std::copy_n(results.begin(), results.size() - 1, std::back_inserter(new_results));
-      return UpdateNumberOfCompleteResultsAndReturn(new_results);
-    }
-    // We have only one result and it is partial
-    if (results.size() == 1) {
-      // check if there is a row change
-      if (partial_results_.at(0)->Row() == last->Row()) {
-        partial_results_.push_back(last);
-        return std::vector<std::shared_ptr<Result>>{};
-      }
-      auto complete_result = Combine();
-      partial_results_.push_back(last);
-      return UpdateNumberOfCompleteResultsAndReturn(complete_result);
-    }
-    // We have some complete results
-    auto results_to_return = PrependCombined(results, results.size() - 1);
-    partial_results_.push_back(last);
-    return UpdateNumberOfCompleteResultsAndReturn(results_to_return);
-  }
-  if (!partial_results_.empty()) {
-    return UpdateNumberOfCompleteResultsAndReturn(PrependCombined(results, results.size()));
-  }
-  return UpdateNumberOfCompleteResultsAndReturn(results);
-}
-
-void ScanResultCache::Clear() { partial_results_.clear(); }
-
-std::shared_ptr<Result> ScanResultCache::CreateCompleteResult(
-    const std::vector<std::shared_ptr<Result>> &partial_results) {
-  if (partial_results.empty()) {
-    return std::make_shared<Result>(std::vector<std::shared_ptr<Cell>>{}, false, false, false);
-  }
-  std::vector<std::shared_ptr<Cell>> cells{};
-  bool stale = false;
-  std::string prev_row = "";
-  std::string current_row = "";
-  size_t i = 0;
-  for (const auto &r : partial_results) {
-    current_row = r->Row();
-    if (i != 0 && prev_row != current_row) {
-      throw new std::runtime_error(
-          "Cannot form complete result. Rows of partial results do not match.");
-    }
-    // Ensure that all Results except the last one are marked as partials. The last result
-    // may not be marked as a partial because Results are only marked as partials when
-    // the scan on the server side must be stopped due to reaching the maxResultSize.
-    // Visualizing it makes it easier to understand:
-    // maxResultSize: 2 cells
-    // (-x-) represents cell number x in a row
-    // Example: row1: -1- -2- -3- -4- -5- (5 cells total)
-    // How row1 will be returned by the server as partial Results:
-    // Result1: -1- -2- (2 cells, size limit reached, mark as partial)
-    // Result2: -3- -4- (2 cells, size limit reached, mark as partial)
-    // Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
-    if (i != partial_results.size() - 1 && !r->Partial()) {
-      throw new std::runtime_error("Cannot form complete result. Result is missing partial flag.");
-    }
-    prev_row = current_row;
-    stale = stale || r->Stale();
-    for (const auto &c : r->Cells()) {
-      cells.push_back(c);
-    }
-    i++;
-  }
-
-  return std::make_shared<Result>(cells, false, stale, false);
-}
-
-std::shared_ptr<Result> ScanResultCache::Combine() {
-  auto result = CreateCompleteResult(partial_results_);
-  partial_results_.clear();
-  return result;
-}
-
-std::vector<std::shared_ptr<Result>> ScanResultCache::PrependCombined(
-    const std::vector<std::shared_ptr<Result>> &results, int length) {
-  if (length == 0) {
-    return std::vector<std::shared_ptr<Result>>{Combine()};
-  }
-  // the last part of a partial result may not be marked as partial so here we need to check if
-  // there is a row change.
-  size_t start;
-  if (partial_results_[0]->Row() == results[0]->Row()) {
-    partial_results_.push_back(results[0]);
-    start = 1;
-    length--;
-  } else {
-    start = 0;
-  }
-  std::vector<std::shared_ptr<Result>> prepend_results{};
-  prepend_results.push_back(Combine());
-  std::copy_n(results.begin() + start, length, std::back_inserter(prepend_results));
-  return prepend_results;
-}
-
-std::vector<std::shared_ptr<Result>> ScanResultCache::UpdateNumberOfCompleteResultsAndReturn(
-    const std::shared_ptr<Result> &result) {
-  return UpdateNumberOfCompleteResultsAndReturn(std::vector<std::shared_ptr<Result>>{result});
-}
-
-std::vector<std::shared_ptr<Result>> ScanResultCache::UpdateNumberOfCompleteResultsAndReturn(
-    const std::vector<std::shared_ptr<Result>> &results) {
-  num_complete_rows_ += results.size();
-  return results;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan-result-cache.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan-result-cache.h b/hbase-native-client/core/scan-result-cache.h
deleted file mode 100644
index 5d3d0ab..0000000
--- a/hbase-native-client/core/scan-result-cache.h
+++ /dev/null
@@ -1,80 +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.
- *
- */
-#pragma once
-
-#include <folly/Logging.h>
-#include <algorithm>
-#include <chrono>
-#include <iterator>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "core/result.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-class ScanResultCache {
-  // In Java, there are 3 different implementations for this. We are not doing partial results,
-  // or scan batching in native code for now, so this version is simpler and
-  // only deals with giving back complete rows as Result. It is more or less implementation
-  // of CompleteScanResultCache.java
-
- public:
-  /**
-   * Add the given results to cache and get valid results back.
-   * @param results the results of a scan next. Must not be null.
-   * @param is_hearthbeat indicate whether the results is gotten from a heartbeat response.
-   * @return valid results, never null.
-   */
-  std::vector<std::shared_ptr<Result>> AddAndGet(
-      const std::vector<std::shared_ptr<Result>> &results, bool is_hearthbeat);
-
-  void Clear();
-
-  int64_t num_complete_rows() const { return num_complete_rows_; }
-
- private:
-  /**
-     * Forms a single result from the partial results in the partialResults list. This method is
-     * useful for reconstructing partial results on the client side.
-     * @param partial_results list of partial results
-     * @return The complete result that is formed by combining all of the partial results together
-     */
-  static std::shared_ptr<Result> CreateCompleteResult(
-      const std::vector<std::shared_ptr<Result>> &partial_results);
-
-  std::shared_ptr<Result> Combine();
-
-  std::vector<std::shared_ptr<Result>> PrependCombined(
-      const std::vector<std::shared_ptr<Result>> &results, int length);
-
-  std::vector<std::shared_ptr<Result>> UpdateNumberOfCompleteResultsAndReturn(
-      const std::shared_ptr<Result> &result);
-
-  std::vector<std::shared_ptr<Result>> UpdateNumberOfCompleteResultsAndReturn(
-      const std::vector<std::shared_ptr<Result>> &results);
-
- private:
-  std::vector<std::shared_ptr<Result>> partial_results_;
-  int64_t num_complete_rows_ = 0;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan-test.cc b/hbase-native-client/core/scan-test.cc
deleted file mode 100644
index 0ee054c..0000000
--- a/hbase-native-client/core/scan-test.cc
+++ /dev/null
@@ -1,228 +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 <gtest/gtest.h>
-#include <limits>
-
-#include "core/scan.h"
-
-using hbase::Get;
-using hbase::Scan;
-
-void CheckFamilies(Scan &scan) {
-  EXPECT_EQ(false, scan.HasFamilies());
-  scan.AddFamily("family-1");
-  EXPECT_EQ(true, scan.HasFamilies());
-  EXPECT_EQ(1, scan.FamilyMap().size());
-  for (const auto &family : scan.FamilyMap()) {
-    EXPECT_STREQ("family-1", family.first.c_str());
-    EXPECT_EQ(0, family.second.size());
-  }
-  // Not allowed to add the same CF.
-  scan.AddFamily("family-1");
-  EXPECT_EQ(1, scan.FamilyMap().size());
-  scan.AddFamily("family-2");
-  EXPECT_EQ(2, scan.FamilyMap().size());
-  scan.AddFamily("family-3");
-  EXPECT_EQ(3, scan.FamilyMap().size());
-  int i = 1;
-  for (const auto &family : scan.FamilyMap()) {
-    std::string family_name = "family-" + std::to_string(i);
-    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
-    EXPECT_EQ(0, family.second.size());
-    i += 1;
-  }
-
-  scan.AddColumn("family-1", "column-1");
-  scan.AddColumn("family-1", "column-2");
-  scan.AddColumn("family-1", "");
-  scan.AddColumn("family-1", "column-3");
-  scan.AddColumn("family-2", "column-X");
-
-  EXPECT_EQ(3, scan.FamilyMap().size());
-  auto it = scan.FamilyMap().begin();
-  EXPECT_STREQ("family-1", it->first.c_str());
-  EXPECT_EQ(4, it->second.size());
-  EXPECT_STREQ("column-1", it->second[0].c_str());
-  EXPECT_STREQ("column-2", it->second[1].c_str());
-  EXPECT_STREQ("", it->second[2].c_str());
-  EXPECT_STREQ("column-3", it->second[3].c_str());
-  ++it;
-  EXPECT_STREQ("family-2", it->first.c_str());
-  EXPECT_EQ(1, it->second.size());
-  EXPECT_STREQ("column-X", it->second[0].c_str());
-  ++it;
-  EXPECT_STREQ("family-3", it->first.c_str());
-  EXPECT_EQ(0, it->second.size());
-  ++it;
-  EXPECT_EQ(it, scan.FamilyMap().end());
-}
-
-void CheckFamiliesAfterCopy(const Scan &scan) {
-  EXPECT_EQ(true, scan.HasFamilies());
-  EXPECT_EQ(3, scan.FamilyMap().size());
-  int i = 1;
-  for (const auto &family : scan.FamilyMap()) {
-    std::string family_name = "family-" + std::to_string(i);
-    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
-    i += 1;
-  }
-  // Check if the alreaday added CF's and CQ's are as expected
-  auto it = scan.FamilyMap().begin();
-  EXPECT_STREQ("family-1", it->first.c_str());
-  EXPECT_EQ(4, it->second.size());
-  EXPECT_STREQ("column-1", it->second[0].c_str());
-  EXPECT_STREQ("column-2", it->second[1].c_str());
-  EXPECT_STREQ("", it->second[2].c_str());
-  EXPECT_STREQ("column-3", it->second[3].c_str());
-  ++it;
-  EXPECT_STREQ("family-2", it->first.c_str());
-  EXPECT_EQ(1, it->second.size());
-  EXPECT_STREQ("column-X", it->second[0].c_str());
-  ++it;
-  EXPECT_STREQ("family-3", it->first.c_str());
-  EXPECT_EQ(0, it->second.size());
-  ++it;
-  EXPECT_EQ(it, scan.FamilyMap().end());
-}
-
-void ScanMethods(Scan &scan) {
-  scan.SetReversed(true);
-  EXPECT_EQ(true, scan.IsReversed());
-  scan.SetReversed(false);
-  EXPECT_EQ(false, scan.IsReversed());
-
-  std::string start_row("start-row");
-  std::string stop_row("stop-row");
-  scan.SetStartRow(start_row);
-  EXPECT_EQ(start_row, scan.StartRow());
-
-  scan.SetStopRow(stop_row);
-  EXPECT_EQ(stop_row, scan.StopRow());
-
-  scan.SetCaching(3);
-  EXPECT_EQ(3, scan.Caching());
-
-  scan.SetConsistency(hbase::pb::Consistency::STRONG);
-  EXPECT_EQ(hbase::pb::Consistency::STRONG, scan.Consistency());
-  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
-  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, scan.Consistency());
-
-  scan.SetCacheBlocks(true);
-  EXPECT_EQ(true, scan.CacheBlocks());
-  scan.SetCacheBlocks(false);
-  EXPECT_EQ(false, scan.CacheBlocks());
-
-  scan.SetAllowPartialResults(true);
-  EXPECT_EQ(true, scan.AllowPartialResults());
-  scan.SetAllowPartialResults(false);
-  EXPECT_EQ(false, scan.AllowPartialResults());
-
-  scan.SetLoadColumnFamiliesOnDemand(true);
-  EXPECT_EQ(true, scan.LoadColumnFamiliesOnDemand());
-  scan.SetLoadColumnFamiliesOnDemand(false);
-  EXPECT_EQ(false, scan.LoadColumnFamiliesOnDemand());
-
-  scan.SetMaxVersions();
-  EXPECT_EQ(1, scan.MaxVersions());
-  scan.SetMaxVersions(20);
-  EXPECT_EQ(20, scan.MaxVersions());
-
-  scan.SetMaxResultSize(1024);
-  EXPECT_EQ(1024, scan.MaxResultSize());
-
-  // Test initial values
-  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
-  EXPECT_EQ(std::numeric_limits<int64_t>::max(), scan.Timerange().MaxTimeStamp());
-
-  // Set & Test new values using TimeRange and TimeStamp
-  scan.SetTimeRange(1000, 2000);
-  EXPECT_EQ(1000, scan.Timerange().MinTimeStamp());
-  EXPECT_EQ(2000, scan.Timerange().MaxTimeStamp());
-  scan.SetTimeStamp(0);
-  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
-  EXPECT_EQ(1, scan.Timerange().MaxTimeStamp());
-
-  // Test some exceptions
-  ASSERT_THROW(scan.SetTimeRange(-1000, 2000), std::runtime_error);
-  ASSERT_THROW(scan.SetTimeRange(1000, -2000), std::runtime_error);
-  ASSERT_THROW(scan.SetTimeRange(1000, 200), std::runtime_error);
-  ASSERT_THROW(scan.SetTimeStamp(std::numeric_limits<int64_t>::max()), std::runtime_error);
-}
-
-TEST(Scan, Object) {
-  Scan scan;
-  ScanMethods(scan);
-  CheckFamilies(scan);
-
-  // Resetting TimeRange values so that the copy construction and assignment
-  // operator tests pass.
-  scan.SetTimeRange(0, std::numeric_limits<int64_t>::max());
-  Scan scancp(scan);
-  ScanMethods(scancp);
-  CheckFamiliesAfterCopy(scancp);
-
-  Scan scaneq;
-  scaneq = scan;
-  ScanMethods(scaneq);
-  CheckFamiliesAfterCopy(scaneq);
-}
-
-TEST(Scan, WithStartRow) {
-  Scan("row-test");
-  Scan scan("row-test");
-  ScanMethods(scan);
-  CheckFamilies(scan);
-}
-
-TEST(Scan, WithStartAndStopRow) {
-  Scan("start-row", "stop-row");
-  Scan scan("start-row", "stop-row");
-  ScanMethods(scan);
-  CheckFamilies(scan);
-}
-
-TEST(Scan, FromGet) {
-  std::string row_str = "row-test";
-  Get get = Get(row_str);
-
-  get.SetCacheBlocks(true);
-  get.SetMaxVersions(5);
-  get.AddFamily("family-1");
-  get.AddFamily("family-1");
-  get.AddFamily("family-2");
-  get.AddFamily("family-3");
-  get.AddColumn("family-1", "column-1");
-  get.AddColumn("family-1", "column-2");
-  get.AddColumn("family-1", "");
-  get.AddColumn("family-1", "column-3");
-  get.AddColumn("family-2", "column-X");
-
-  EXPECT_EQ(3, get.FamilyMap().size());
-
-  Scan scan(get);
-  ScanMethods(scan);
-  CheckFamiliesAfterCopy(scan);
-}
-
-TEST(Scan, Exception) {
-  std::string row(std::numeric_limits<int16_t>::max() + 1, 'X');
-  ASSERT_THROW(Scan tmp(row), std::runtime_error);
-  ASSERT_THROW(Scan tmp(""), std::runtime_error);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.cc b/hbase-native-client/core/scan.cc
deleted file mode 100644
index 6dcc51b..0000000
--- a/hbase-native-client/core/scan.cc
+++ /dev/null
@@ -1,186 +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/scan.h"
-
-#include <algorithm>
-#include <iterator>
-#include <limits>
-#include <stdexcept>
-
-namespace hbase {
-
-Scan::Scan() {}
-
-Scan::~Scan() {}
-
-Scan::Scan(const std::string &start_row) : start_row_(start_row) { CheckRow(start_row_); }
-
-Scan::Scan(const std::string &start_row, const std::string &stop_row)
-    : start_row_(start_row), stop_row_(stop_row) {
-  CheckRow(start_row_);
-  CheckRow(stop_row_);
-}
-
-Scan::Scan(const Scan &scan) : Query(scan) {
-  start_row_ = scan.start_row_;
-  stop_row_ = scan.stop_row_;
-  max_versions_ = scan.max_versions_;
-  caching_ = scan.caching_;
-  max_result_size_ = scan.max_result_size_;
-  cache_blocks_ = scan.cache_blocks_;
-  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
-  reversed_ = scan.reversed_;
-  allow_partial_results_ = scan.allow_partial_results_;
-  consistency_ = scan.consistency_;
-  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
-  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
-}
-
-Scan &Scan::operator=(const Scan &scan) {
-  Query::operator=(scan);
-  start_row_ = scan.start_row_;
-  stop_row_ = scan.stop_row_;
-  max_versions_ = scan.max_versions_;
-  caching_ = scan.caching_;
-  max_result_size_ = scan.max_result_size_;
-  cache_blocks_ = scan.cache_blocks_;
-  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
-  reversed_ = scan.reversed_;
-  allow_partial_results_ = scan.allow_partial_results_;
-  consistency_ = scan.consistency_;
-  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
-  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
-  return *this;
-}
-
-Scan::Scan(const Get &get) {
-  cache_blocks_ = get.CacheBlocks();
-  max_versions_ = get.MaxVersions();
-  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
-  family_map_.insert(get.FamilyMap().begin(), get.FamilyMap().end());
-}
-
-Scan &Scan::AddFamily(const std::string &family) {
-  const auto &it = family_map_.find(family);
-  /**
-   * Check if any qualifiers are already present or not.
-   * Remove all existing qualifiers if the given family is already present in
-   * the map
-   */
-  if (family_map_.end() != it) {
-    it->second.clear();
-  } else {
-    family_map_[family];
-  }
-  return *this;
-}
-
-Scan &Scan::AddColumn(const std::string &family, const std::string &qualifier) {
-  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
-  /**
-   * Check if any qualifiers are already present or not.
-   * Add only if qualifiers for a given family are not present
-   */
-  if (it == family_map_[family].end()) {
-    family_map_[family].push_back(qualifier);
-  }
-  return *this;
-}
-
-void Scan::SetReversed(bool reversed) { reversed_ = reversed; }
-
-bool Scan::IsReversed() const { return reversed_; }
-
-void Scan::SetStartRow(const std::string &start_row) { start_row_ = start_row; }
-
-const std::string &Scan::StartRow() const { return start_row_; }
-
-void Scan::SetStopRow(const std::string &stop_row) { stop_row_ = stop_row; }
-
-const std::string &Scan::StopRow() const { return stop_row_; }
-
-void Scan::SetCaching(int caching) { caching_ = caching; }
-
-int Scan::Caching() const { return caching_; }
-
-Scan &Scan::SetConsistency(const hbase::pb::Consistency consistency) {
-  consistency_ = consistency;
-  return *this;
-}
-
-hbase::pb::Consistency Scan::Consistency() const { return consistency_; }
-
-void Scan::SetCacheBlocks(bool cache_blocks) { cache_blocks_ = cache_blocks; }
-
-bool Scan::CacheBlocks() const { return cache_blocks_; }
-
-void Scan::SetAllowPartialResults(bool allow_partial_results) {
-  allow_partial_results_ = allow_partial_results;
-}
-
-bool Scan::AllowPartialResults() const { return allow_partial_results_; }
-
-void Scan::SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand) {
-  load_column_families_on_demand_ = load_column_families_on_demand;
-}
-
-bool Scan::LoadColumnFamiliesOnDemand() const { return load_column_families_on_demand_; }
-
-Scan &Scan::SetMaxVersions(uint32_t max_versions) {
-  max_versions_ = max_versions;
-  return *this;
-}
-
-int Scan::MaxVersions() const { return max_versions_; }
-
-void Scan::SetMaxResultSize(int64_t max_result_size) { max_result_size_ = max_result_size; }
-
-int64_t Scan::MaxResultSize() const { return max_result_size_; }
-
-Scan &Scan::SetTimeRange(int64_t min_stamp, int64_t max_stamp) {
-  tr_.reset(new TimeRange(min_stamp, max_stamp));
-  return *this;
-}
-
-Scan &Scan::SetTimeStamp(int64_t timestamp) {
-  tr_.reset(new TimeRange(timestamp, timestamp + 1));
-  return *this;
-}
-
-const TimeRange &Scan::Timerange() const { return *tr_; }
-
-void Scan::CheckRow(const std::string &row) {
-  const int32_t kMaxRowLength = std::numeric_limits<int16_t>::max();
-  int row_length = row.size();
-  if (0 == row_length) {
-    throw std::runtime_error("Row length can't be 0");
-  }
-  if (row_length > kMaxRowLength) {
-    throw std::runtime_error("Length of " + row + " is greater than max row size: " +
-                             std::to_string(kMaxRowLength));
-  }
-}
-
-bool Scan::HasFamilies() const { return !family_map_.empty(); }
-
-const std::map<std::string, std::vector<std::string>> &Scan::FamilyMap() const {
-  return family_map_;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scan.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.h b/hbase-native-client/core/scan.h
deleted file mode 100644
index 1085c4b..0000000
--- a/hbase-native-client/core/scan.h
+++ /dev/null
@@ -1,270 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/get.h"
-#include "core/time-range.h"
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-class Scan : public Query {
- public:
-  /**
-   * @brief Constructors. Create a Scan operation across all rows.
-   */
-  Scan();
-  Scan(const Scan &scan);
-  Scan &operator=(const Scan &scan);
-
-  ~Scan();
-
-  /**
-   * @brief Create a Scan operation starting at the specified row. If the
-   * specified row does not exist,
-   * the Scanner will start from the next closest row after the specified row.
-   * @param start_row - row to start scanner at or after
-   */
-  explicit Scan(const std::string &start_row);
-
-  /**
-   * @brief Create a Scan operation for the range of rows specified.
-   * @param start_row - row to start scanner at or after (inclusive).
-   * @param stop_row - row to stop scanner before (exclusive).
-   */
-  Scan(const std::string &start_row, const std::string &stop_row);
-
-  /**
-   * @brief Builds a scan object with the same specs as get.
-   * @param get - get to model scan after
-   */
-  explicit Scan(const Get &get);
-
-  /**
-   * @brief Get all columns from the specified family.Overrides previous calls
-   * to AddColumn for this family.
-   * @param family - family name
-   */
-  Scan &AddFamily(const std::string &family);
-
-  /**
-   * @brief Get the column from the specified family with the specified
-   * qualifier.Overrides previous calls to AddFamily for this family.
-   * @param family - family name.
-   * @param qualifier - column qualifier.
-   */
-  Scan &AddColumn(const std::string &family, const std::string &qualifier);
-
-  /**
-   * @brief Set whether this scan is a reversed one. This is false by default
-   * which means forward(normal) scan.
-   * @param reversed - if true, scan will be backward order
-   */
-  void SetReversed(bool reversed);
-
-  /**
-   * @brief Get whether this scan is a reversed one. Returns  true if backward
-   * scan, false if forward(default) scan
-   */
-  bool IsReversed() const;
-
-  /**
-   * @brief Set the start row of the scan.If the specified row does not exist,
-   * the Scanner will start from the next closest row after the specified row.
-   * @param start_row - row to start scanner at or after
-   * @throws std::runtime_error if start_row length is 0 or greater than
-   * MAX_ROW_LENGTH
-   */
-  void SetStartRow(const std::string &start_row);
-
-  /**
-   * @brief returns start_row of the Scan.
-   */
-  const std::string &StartRow() const;
-
-  /**
-   * @brief Set the stop row of the scan. The scan will include rows that are
-   * lexicographically less than the provided stop_row.
-   * @param stop_row - row to end at (exclusive)
-   * @throws std::runtime_error if stop_row length is 0 or greater than
-   * MAX_ROW_LENGTH
-   */
-  void SetStopRow(const std::string &stop_row);
-
-  /**
-   * @brief returns stop_row of the Scan.
-   */
-  const std::string &StopRow() const;
-
-  /**
-   * @brief Set the number of rows for caching that will be passed to scanners.
-   * Higher caching values will enable faster scanners but will use more memory.
-   * @param caching - the number of rows for caching.
-   */
-  void SetCaching(int caching);
-
-  /**
-   * @brief caching the number of rows fetched when calling next on a scanner.
-   */
-  int Caching() const;
-
-  /**
-   * @brief Sets the consistency level for this operation.
-   * @param consistency - the consistency level
-   */
-  Scan &SetConsistency(const hbase::pb::Consistency consistency);
-
-  /**
-   * @brief Returns the consistency level for this operation.
-   */
-  hbase::pb::Consistency Consistency() const;
-
-  /**
-   * @brief Set whether blocks should be cached for this Scan.This is true by
-   * default. When true, default settings of the table and family are used (this
-   * will never override caching blocks if the block cache is disabled for that
-   * family or entirely).
-   * @param cache_blocks - if false, default settings are overridden and blocks
-   * will not be cached
-   */
-  void SetCacheBlocks(bool cache_blocks);
-
-  /**
-   * @brief Get whether blocks should be cached for this Scan.
-   */
-  bool CacheBlocks() const;
-
-  /**
-   * @brief Setting whether the caller wants to see the partial results that may
-   * be returned from the server. By default this value is false and the
-   * complete results will be assembled client side before being delivered to
-   * the caller.
-   * @param allow_partial_results - if true partial results will be returned.
-   */
-  void SetAllowPartialResults(bool allow_partial_results);
-
-  /**
-   * @brief true when the constructor of this scan understands that the results
-   * they will see may only represent a partial portion of a row. The entire row
-   * would be retrieved by subsequent calls to ResultScanner.next()
-   */
-  bool AllowPartialResults() const;
-
-  /**
-   * @brief Set the value indicating whether loading CFs on demand should be
-   * allowed (cluster default is false). On-demand CF loading doesn't load
-   * column families until necessary.
-   * @param load_column_families_on_demand
-   */
-  void SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand);
-
-  /**
-   * @brief Get the raw loadColumnFamiliesOnDemand setting.
-   */
-  bool LoadColumnFamiliesOnDemand() const;
-
-  /**
-   * @brief Get up to the specified number of versions of each column if
-   * specified else get default i.e. one.
-   * @param max_versions - maximum versions for each column.
-   */
-  Scan &SetMaxVersions(uint32_t max_versions = 1);
-
-  /**
-   * @brief the max number of versions to fetch
-   */
-  int MaxVersions() const;
-
-  /**
-   * @brief Set the maximum result size. The default is -1; this means that no
-   * specific maximum result size will be set for this scan, and the global
-   * configured value will be used instead. (Defaults to unlimited).
-   * @param The maximum result size in bytes.
-   */
-  void SetMaxResultSize(int64_t max_result_size);
-
-  /**
-   * @brief the maximum result size in bytes.
-   */
-  int64_t MaxResultSize() const;
-
-  /**
-   * @brief Get versions of columns only within the specified timestamp range,
-   * [min_stamp, max_stamp). Note, default maximum versions to return is 1. If
-   * your time range spans more than one version and you want all versions
-   * returned, up the number of versions beyond the default.
-   * @param min_stamp - minimum timestamp value, inclusive.
-   * @param max_stamp - maximum timestamp value, exclusive.
-   */
-  Scan &SetTimeRange(int64_t min_stamp, int64_t max_stamp);
-
-  /**
-   * @brief Get versions of columns with the specified timestamp. Note, default
-   * maximum versions to return is 1. If your time range spans more than one
-   * version and you want all versions returned, up the number of versions
-   * beyond the defaut.
-   * @param timestamp - version timestamp
-   */
-  Scan &SetTimeStamp(int64_t timestamp);
-
-  /**
-   * @brief Return Timerange
-   */
-  const TimeRange &Timerange() const;
-
-  /**
-   * @brief Returns true if family map is non empty false otherwise
-   */
-  bool HasFamilies() const;
-
-  /**
-   * @brief Returns the Scan family map for this Scan operation.
-   */
-  const std::map<std::string, std::vector<std::string>> &FamilyMap() const;
-
- private:
-  std::string start_row_ = "";
-  std::string stop_row_ = "";
-  uint32_t max_versions_ = 1;
-  int32_t caching_ = -1;
-  int64_t max_result_size_ = -1;
-  bool cache_blocks_ = true;
-  bool load_column_families_on_demand_ = false;
-  bool reversed_ = false;
-  bool allow_partial_results_ = false;
-  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
-  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
-  std::map<std::string, std::vector<std::string>> family_map_;
-
-  /**
-   * @brief Checks for row length validity, throws if length check fails,
-   * returns null otherwise.
-   * @param row - row whose validity needs to be checked
-   * @throws std::runtime_error if row length equals 0 or greater than
-   * std::numeric_limits<short>::max();
-   */
-  void CheckRow(const std::string &row);
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/scanner-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scanner-test.cc b/hbase-native-client/core/scanner-test.cc
deleted file mode 100644
index 1ecbd02..0000000
--- a/hbase-native-client/core/scanner-test.cc
+++ /dev/null
@@ -1,368 +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 <folly/Conv.h>
-#include <gtest/gtest.h>
-
-#include <chrono>
-#include <thread>
-#include <vector>
-
-#include "core/async-client-scanner.h"
-#include "core/async-table-result-scanner.h"
-#include "core/cell.h"
-#include "core/client.h"
-#include "core/configuration.h"
-#include "core/filter.h"
-#include "core/get.h"
-#include "core/hbase-configuration-loader.h"
-#include "core/put.h"
-#include "core/result.h"
-#include "core/row.h"
-#include "core/table.h"
-#include "if/Comparator.pb.h"
-#include "if/Filter.pb.h"
-#include "serde/table-name.h"
-#include "test-util/test-util.h"
-#include "utils/time-util.h"
-
-using hbase::Cell;
-using hbase::ComparatorFactory;
-using hbase::Comparator;
-using hbase::Configuration;
-using hbase::Get;
-using hbase::Put;
-using hbase::Result;
-using hbase::Scan;
-using hbase::Table;
-using hbase::TestUtil;
-using hbase::TimeUtil;
-using hbase::AsyncClientScanner;
-using hbase::AsyncTableResultScanner;
-using hbase::FilterFactory;
-using hbase::pb::CompareType;
-
-class ScannerTest : public ::testing::Test {
- public:
-  static std::unique_ptr<hbase::TestUtil> test_util;
-  static const uint32_t num_rows;
-
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util = std::make_unique<hbase::TestUtil>();
-    test_util->StartMiniCluster(2);
-  }
-};
-std::unique_ptr<hbase::TestUtil> ScannerTest::test_util = nullptr;
-const uint32_t ScannerTest::num_rows = 1000;
-
-std::string Family(uint32_t i) { return "f" + folly::to<std::string>(i); }
-
-std::string Row(uint32_t i, int width) {
-  std::ostringstream s;
-  s.fill('0');
-  s.width(width);
-  s << i;
-  return "row" + s.str();
-}
-
-std::string Row(uint32_t i) { return Row(i, 3); }
-
-std::unique_ptr<Put> MakePut(const std::string &row, uint32_t num_families) {
-  auto put = std::make_unique<Put>(row);
-
-  for (uint32_t i = 0; i < num_families; i++) {
-    put->AddColumn(Family(i), "q1", row);
-    put->AddColumn(Family(i), "q2", row + "-" + row);
-  }
-
-  return std::move(put);
-}
-
-void CheckResult(const Result &r, std::string expected_row, uint32_t num_families) {
-  VLOG(1) << r.DebugString();
-  auto row = r.Row();
-  ASSERT_EQ(row, expected_row);
-  ASSERT_EQ(r.Cells().size(), num_families * 2);
-  for (uint32_t i = 0; i < num_families; i++) {
-    ASSERT_EQ(*r.Value(Family(i), "q1"), row);
-    ASSERT_EQ(*r.Value(Family(i), "q2"), row + "-" + row);
-  }
-}
-
-void CreateTable(std::string table_name, uint32_t num_families, uint32_t num_rows,
-                 int32_t num_regions) {
-  LOG(INFO) << "Creating the table " << table_name
-            << " with num_regions:" << folly::to<std::string>(num_regions);
-  std::vector<std::string> families;
-  for (uint32_t i = 0; i < num_families; i++) {
-    families.push_back(Family(i));
-  }
-  if (num_regions <= 1) {
-    ScannerTest::test_util->CreateTable(table_name, families);
-  } else {
-    std::vector<std::string> keys;
-    for (int32_t i = 0; i < num_regions - 1; i++) {
-      keys.push_back(Row(i * (num_rows / (num_regions - 1))));
-      LOG(INFO) << "Split key:" << keys[keys.size() - 1];
-    }
-    ScannerTest::test_util->CreateTable(table_name, families, keys);
-  }
-}
-
-std::unique_ptr<hbase::Client> CreateTableAndWriteData(std::string table_name,
-                                                       uint32_t num_families, uint32_t num_rows,
-                                                       int32_t num_regions) {
-  CreateTable(table_name, num_families, num_rows, num_regions);
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-  auto client = std::make_unique<hbase::Client>(*ScannerTest::test_util->conf());
-  auto table = client->Table(tn);
-
-  LOG(INFO) << "Writing data to the table, num_rows:" << num_rows;
-  // Perform Puts
-  for (uint32_t i = 0; i < num_rows; i++) {
-    table->Put(*MakePut(Row(i), num_families));
-  }
-  return std::move(client);
-}
-
-void TestScan(const Scan &scan, uint32_t num_families, int32_t start, int32_t num_rows,
-              Table *table) {
-  LOG(INFO) << "Starting scan for the test with start:" << scan.StartRow()
-            << ", stop:" << scan.StopRow() << " expected_num_rows:" << num_rows;
-  auto scanner = table->Scan(scan);
-
-  uint32_t i = start;
-  auto r = scanner->Next();
-  while (r != nullptr) {
-    CheckResult(*r, Row(i++), num_families);
-    r = scanner->Next();
-  }
-  ASSERT_EQ(i - start, num_rows);
-}
-
-void TestScan(const Scan &scan, int32_t start, int32_t num_rows, Table *table) {
-  TestScan(scan, 1, start, num_rows, table);
-}
-
-void TestScan(uint32_t num_families, int32_t start, int32_t stop, int32_t num_rows, Table *table) {
-  Scan scan{};
-  if (start >= 0) {
-    scan.SetStartRow(Row(start));
-  } else {
-    start = 0;  // neded for below logic
-  }
-  if (stop >= 0) {
-    scan.SetStopRow(Row(stop));
-  }
-
-  TestScan(scan, num_families, start, num_rows, table);
-}
-
-void TestScan(int32_t start, int32_t stop, int32_t num_rows, Table *table) {
-  TestScan(1, start, stop, num_rows, table);
-}
-
-void TestScan(uint32_t num_families, std::string start, std::string stop, int32_t num_rows,
-              Table *table) {
-  Scan scan{};
-
-  scan.SetStartRow(start);
-  scan.SetStopRow(stop);
-
-  LOG(INFO) << "Starting scan for the test with start:" << start << ", stop:" << stop
-            << " expected_num_rows:" << num_rows;
-  auto scanner = table->Scan(scan);
-
-  uint32_t i = 0;
-  auto r = scanner->Next();
-  while (r != nullptr) {
-    VLOG(1) << r->DebugString();
-    i++;
-    ASSERT_EQ(r->Map().size(), num_families);
-    r = scanner->Next();
-  }
-  ASSERT_EQ(i, num_rows);
-}
-
-void TestScan(std::string start, std::string stop, int32_t num_rows, Table *table) {
-  TestScan(1, start, stop, num_rows, table);
-}
-
-void TestScanCombinations(Table *table, uint32_t num_families) {
-  // full table
-  TestScan(num_families, -1, -1, 1000, table);
-  TestScan(num_families, -1, 999, 999, table);
-  TestScan(num_families, 0, -1, 1000, table);
-  TestScan(num_families, 0, 999, 999, table);
-  TestScan(num_families, 10, 990, 980, table);
-  TestScan(num_families, 1, 998, 997, table);
-
-  TestScan(num_families, 123, 345, 222, table);
-  TestScan(num_families, 234, 456, 222, table);
-  TestScan(num_families, 345, 567, 222, table);
-  TestScan(num_families, 456, 678, 222, table);
-
-  // single results
-  TestScan(num_families, 111, 111, 1, table);  // split keys are like 111, 222, 333, etc
-  TestScan(num_families, 111, 112, 1, table);
-  TestScan(num_families, 332, 332, 1, table);
-  TestScan(num_families, 332, 333, 1, table);
-  TestScan(num_families, 333, 333, 1, table);
-  TestScan(num_families, 333, 334, 1, table);
-  TestScan(num_families, 42, 42, 1, table);
-  TestScan(num_families, 921, 921, 1, table);
-  TestScan(num_families, 0, 0, 1, table);
-  TestScan(num_families, 0, 1, 1, table);
-  TestScan(num_families, 999, 999, 1, table);
-
-  // few results
-  TestScan(num_families, 0, 0, 1, table);
-  TestScan(num_families, 0, 2, 2, table);
-  TestScan(num_families, 0, 5, 5, table);
-  TestScan(num_families, 10, 15, 5, table);
-  TestScan(num_families, 105, 115, 10, table);
-  TestScan(num_families, 111, 221, 110, table);
-  TestScan(num_families, 111, 222, 111, table);  // crossing region boundary 111-222
-  TestScan(num_families, 111, 223, 112, table);
-  TestScan(num_families, 111, 224, 113, table);
-  TestScan(num_families, 990, 999, 9, table);
-  TestScan(num_families, 900, 998, 98, table);
-
-  // empty results
-  TestScan(num_families, "a", "a", 0, table);
-  TestScan(num_families, "a", "r", 0, table);
-  TestScan(num_families, "", "r", 0, table);
-  TestScan(num_families, "s", "", 0, table);
-  TestScan(num_families, "s", "z", 0, table);
-  TestScan(num_families, Row(110) + "a", Row(111), 0, table);
-  TestScan(num_families, Row(111) + "a", Row(112), 0, table);
-  TestScan(num_families, Row(123) + "a", Row(124), 0, table);
-
-  // custom
-  TestScan(num_families, Row(111, 3), Row(1111, 4), 1, table);
-  TestScan(num_families, Row(0, 3), Row(0, 4), 1, table);
-  TestScan(num_families, Row(999, 3), Row(9999, 4), 1, table);
-  TestScan(num_families, Row(111, 3), Row(1111, 4), 1, table);
-  TestScan(num_families, Row(0, 3), Row(9999, 4), 1000, table);
-  TestScan(num_families, "a", "z", 1000, table);
-}
-
-// some of these tests are from TestAsyncTableScan* and some from TestFromClientSide* and
-// TestScannersFromClientSide*
-
-TEST_F(ScannerTest, SingleRegionScan) {
-  auto client = CreateTableAndWriteData("t_single_region_scan", 1, num_rows, 1);
-  auto table = client->Table(folly::to<hbase::pb::TableName>("t_single_region_scan"));
-
-  TestScanCombinations(table.get(), 1);
-}
-
-TEST_F(ScannerTest, MultiRegionScan) {
-  auto client = CreateTableAndWriteData("t_multi_region_scan", 1, num_rows, 10);
-  auto table = client->Table(folly::to<hbase::pb::TableName>("t_multi_region_scan"));
-
-  TestScanCombinations(table.get(), 1);
-}
-
-TEST_F(ScannerTest, ScanWithPauses) {
-  auto max_result_size =
-      ScannerTest::test_util->conf()->GetInt("hbase.client.scanner.max.result.size", 2097152);
-  ScannerTest::test_util->conf()->SetInt("hbase.client.scanner.max.result.size", 100);
-  auto client = CreateTableAndWriteData("t_multi_region_scan", 1, num_rows, 5);
-  auto table = client->Table(folly::to<hbase::pb::TableName>("t_multi_region_scan"));
-
-  VLOG(1) << "Starting scan for the test";
-  Scan scan{};
-  scan.SetCaching(100);
-  auto scanner = table->Scan(scan);
-
-  uint32_t i = 0;
-  auto r = scanner->Next();
-  while (r != nullptr) {
-    CheckResult(*r, Row(i++), 1);
-    r = scanner->Next();
-    std::this_thread::sleep_for(TimeUtil::MillisToNanos(10));
-  }
-
-  auto s = static_cast<AsyncTableResultScanner *>(scanner.get());
-  ASSERT_GT(s->num_prefetch_stopped(), 0);
-
-  ASSERT_EQ(i, num_rows);
-  ScannerTest::test_util->conf()->SetInt("hbase.client.scanner.max.result.size", max_result_size);
-}
-
-TEST_F(ScannerTest, ScanWithFilters) {
-  auto client = CreateTableAndWriteData("t_scan_with_filters", 1, num_rows, 1);
-  auto table = client->Table(folly::to<hbase::pb::TableName>("t_scan_with_filters"));
-
-  Scan scan{};
-  scan.SetFilter(FilterFactory::ValueFilter(CompareType::GREATER_OR_EQUAL,
-                                            *ComparatorFactory::BinaryComparator(Row(800))));
-
-  TestScan(scan, 800, 200, table.get());
-}
-
-TEST_F(ScannerTest, ScanMultiFamily) {
-  auto client = CreateTableAndWriteData("t_scan_multi_family", 3, num_rows, 1);
-  auto table = client->Table(folly::to<hbase::pb::TableName>("t_scan_multi_family"));
-
-  TestScanCombinations(table.get(), 3);
-}
-
-TEST_F(ScannerTest, ScanNullQualifier) {
-  std::string table_name{"t_scan_null_qualifier"};
-  std::string row{"row"};
-  CreateTable(table_name, 1, 1, 1);
-
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-  auto client = std::make_unique<hbase::Client>(*ScannerTest::test_util->conf());
-  auto table = client->Table(tn);
-
-  // Perform Puts
-  Put put{row};
-  put.AddColumn(Family(0), "q1", row);
-  put.AddColumn(Family(0), "", row);
-  table->Put(put);
-
-  Scan scan1{};
-  scan1.AddColumn(Family(0), "");
-  auto scanner1 = table->Scan(scan1);
-  auto r1 = scanner1->Next();
-  ASSERT_EQ(r1->Cells().size(), 1);
-  ASSERT_EQ(scanner1->Next(), nullptr);
-
-  Scan scan2{};
-  scan2.AddFamily(Family(0));
-  auto scanner2 = table->Scan(scan2);
-  auto r2 = scanner2->Next();
-  ASSERT_EQ(r2->Cells().size(), 2);
-  ASSERT_EQ(scanner2->Next(), nullptr);
-}
-
-TEST_F(ScannerTest, ScanNoResults) {
-  std::string table_name{"t_scan_no_results"};
-  auto client = CreateTableAndWriteData(table_name, 1, num_rows, 3);
-  auto table = client->Table(folly::to<hbase::pb::TableName>(table_name));
-
-  Scan scan{};
-  scan.AddColumn(Family(0), "non_existing_qualifier");
-
-  TestScan(scan, 0, 0, table.get());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/server-request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/server-request.h b/hbase-native-client/core/server-request.h
deleted file mode 100644
index 85df9ed..0000000
--- a/hbase-native-client/core/server-request.h
+++ /dev/null
@@ -1,62 +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.
- *
- */
-
-#pragma once
-
-#include <map>
-#include <memory>
-#include <stdexcept>
-#include <string>
-#include "core/action.h"
-#include "core/region-location.h"
-#include "core/region-request.h"
-
-namespace hbase {
-
-class ServerRequest {
- public:
-  // Concurrent
-  using ActionsByRegion = std::map<std::string, std::shared_ptr<RegionRequest>>;
-
-  explicit ServerRequest(std::shared_ptr<RegionLocation> region_location) {
-    auto region_name = region_location->region_name();
-    auto region_request = std::make_shared<RegionRequest>(region_location);
-    actions_by_region_[region_name] = region_request;
-  }
-  ~ServerRequest() {}
-
-  void AddActionsByRegion(std::shared_ptr<RegionLocation> region_location,
-                          std::shared_ptr<Action> action) {
-    auto region_name = region_location->region_name();
-    auto search = actions_by_region_.find(region_name);
-    if (search == actions_by_region_.end()) {
-      auto region_request = std::make_shared<RegionRequest>(region_location);
-      actions_by_region_[region_name] = region_request;
-      actions_by_region_[region_name]->AddAction(action);
-    } else {
-      search->second->AddAction(action);
-    }
-  }
-
-  const ActionsByRegion &actions_by_region() const { return actions_by_region_; }
-
- private:
-  ActionsByRegion actions_by_region_;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/simple-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/simple-client.cc b/hbase-native-client/core/simple-client.cc
deleted file mode 100644
index 6730248..0000000
--- a/hbase-native-client/core/simple-client.cc
+++ /dev/null
@@ -1,207 +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 <folly/Logging.h>
-#include <folly/Random.h>
-#include <gflags/gflags.h>
-
-#include <atomic>
-#include <chrono>
-#include <iostream>
-#include <thread>
-
-#include "connection/rpc-client.h"
-#include "core/client.h"
-#include "core/get.h"
-#include "core/hbase-configuration-loader.h"
-#include "core/put.h"
-#include "core/scan.h"
-#include "core/table.h"
-#include "serde/server-name.h"
-#include "serde/table-name.h"
-#include "utils/time-util.h"
-
-using hbase::Client;
-using hbase::Configuration;
-using hbase::Get;
-using hbase::HBaseConfigurationLoader;
-using hbase::Scan;
-using hbase::Put;
-using hbase::Result;
-using hbase::Table;
-using hbase::pb::TableName;
-using hbase::pb::ServerName;
-using hbase::TimeUtil;
-
-DEFINE_string(table, "test_table", "What table to do the reads or writes");
-DEFINE_string(row, "row_", "row prefix");
-DEFINE_string(zookeeper, "localhost:2181", "What zk quorum to talk to");
-DEFINE_string(conf, "", "Conf directory to read the config from (optional)");
-DEFINE_uint64(num_rows, 10000, "How many rows to write and read");
-DEFINE_uint64(batch_num_rows, 10000, "How many rows batch for multi-gets and multi-puts");
-DEFINE_uint64(report_num_rows, 10000, "How frequent we should report the progress");
-DEFINE_bool(puts, true, "Whether to perform puts");
-DEFINE_bool(gets, true, "Whether to perform gets");
-DEFINE_bool(multigets, true, "Whether to perform multi-gets");
-DEFINE_bool(scans, true, "Whether to perform scans");
-DEFINE_bool(display_results, false, "Whether to display the Results from Gets");
-DEFINE_int32(threads, 6, "How many cpu threads");
-
-std::unique_ptr<Put> MakePut(const std::string &row) {
-  auto put = std::make_unique<Put>(row);
-  put->AddColumn("f", "q", row);
-  return std::move(put);
-}
-
-std::string Row(const std::string &prefix, uint64_t i) {
-  auto suf = folly::to<std::string>(i);
-  return prefix + suf;
-}
-
-void ValidateResult(const Result &result, const std::string &row) {
-  CHECK(!result.IsEmpty());
-  CHECK_EQ(result.Row(), row);
-  CHECK_EQ(result.Size(), 1);
-  CHECK_EQ(result.Value("f", "q").value(), row);
-}
-
-int main(int argc, char *argv[]) {
-  gflags::SetUsageMessage("Simple client to get a single row from HBase on the comamnd line");
-  gflags::ParseCommandLineFlags(&argc, &argv, true);
-  google::InitGoogleLogging(argv[0]);
-  google::InstallFailureSignalHandler();
-  FLAGS_logtostderr = 1;
-  FLAGS_stderrthreshold = 1;
-
-  std::shared_ptr<Configuration> conf = nullptr;
-  if (FLAGS_conf == "") {
-    // Configuration
-    conf = std::make_shared<Configuration>();
-    conf->Set("hbase.zookeeper.quorum", FLAGS_zookeeper);
-    conf->SetInt("hbase.client.cpu.thread.pool.size", FLAGS_threads);
-  } else {
-    setenv("HBASE_CONF", FLAGS_conf.c_str(), 1);
-    hbase::HBaseConfigurationLoader loader;
-    conf = std::make_shared<Configuration>(loader.LoadDefaultResources().value());
-  }
-
-  auto row = FLAGS_row;
-
-  auto tn = std::make_shared<TableName>(folly::to<TableName>(FLAGS_table));
-  auto num_puts = FLAGS_num_rows;
-
-  auto client = std::make_unique<Client>(*conf);
-  auto table = client->Table(*tn);
-
-  auto start_ns = TimeUtil::GetNowNanos();
-
-  // Do the Put requests
-  if (FLAGS_puts) {
-    LOG(INFO) << "Sending put requests";
-    for (uint64_t i = 0; i < num_puts; i++) {
-      table->Put(*MakePut(Row(FLAGS_row, i)));
-      if (i != 0 && i % FLAGS_report_num_rows == 0) {
-        LOG(INFO) << "Sent  " << i << " Put requests in " << TimeUtil::ElapsedMillis(start_ns)
-                  << " ms.";
-      }
-    }
-
-    LOG(INFO) << "Successfully sent  " << num_puts << " Put requests in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-
-  // Do the Get requests
-  if (FLAGS_gets) {
-    LOG(INFO) << "Sending get requests";
-    start_ns = TimeUtil::GetNowNanos();
-    for (uint64_t i = 0; i < num_puts; i++) {
-      auto row = Row(FLAGS_row, i);
-      auto result = table->Get(Get{row});
-      if (FLAGS_display_results) {
-        LOG(INFO) << result->DebugString();
-      } else if (i != 0 && i % FLAGS_report_num_rows == 0) {
-        LOG(INFO) << "Sent  " << i << " Get requests in " << TimeUtil::ElapsedMillis(start_ns)
-                  << " ms.";
-      }
-      ValidateResult(*result, row);
-    }
-
-    LOG(INFO) << "Successfully sent  " << num_puts << " Get requests in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-
-  // Do the Multi-Gets
-  if (FLAGS_multigets) {
-    LOG(INFO) << "Sending multi-get requests";
-    start_ns = TimeUtil::GetNowNanos();
-    std::vector<hbase::Get> gets;
-
-    for (uint64_t i = 0; i < num_puts;) {
-      gets.clear();
-      // accumulate batch_num_rows at a time
-      for (uint64_t j = 0; j < FLAGS_batch_num_rows && i < num_puts; ++j) {
-        hbase::Get get(Row(FLAGS_row, i));
-        gets.push_back(get);
-        i++;
-      }
-      auto results = table->Get(gets);
-
-      if (FLAGS_display_results) {
-        for (const auto &result : results) LOG(INFO) << result->DebugString();
-      } else if (i != 0 && i % FLAGS_report_num_rows == 0) {
-        LOG(INFO) << "Sent  " << i << " Multi-Get requests in " << TimeUtil::ElapsedMillis(start_ns)
-                  << " ms.";
-      }
-    }
-
-    LOG(INFO) << "Successfully sent  " << num_puts << " Multi-Get requests in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-
-  // Do the Scan
-  if (FLAGS_scans) {
-    LOG(INFO) << "Starting scanner";
-    start_ns = TimeUtil::GetNowNanos();
-    Scan scan{};
-    auto scanner = table->Scan(scan);
-
-    uint64_t i = 0;
-    auto r = scanner->Next();
-    while (r != nullptr) {
-      if (FLAGS_display_results) {
-        LOG(INFO) << r->DebugString();
-      }
-      r = scanner->Next();
-      i++;
-      if (!FLAGS_display_results && i != 0 && i % FLAGS_report_num_rows == 0) {
-        LOG(INFO) << "Scan iterated over " << i << " results " << TimeUtil::ElapsedMillis(start_ns)
-                  << " ms.";
-      }
-    }
-
-    LOG(INFO) << "Successfully iterated over  " << i << " Scan results in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-    scanner->Close();
-  }
-
-  table->Close();
-  client->Close();
-
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/table.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/table.cc b/hbase-native-client/core/table.cc
deleted file mode 100644
index f93a029..0000000
--- a/hbase-native-client/core/table.cc
+++ /dev/null
@@ -1,144 +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.h"
-
-#include <chrono>
-#include <limits>
-#include <utility>
-#include <vector>
-
-#include "core/async-connection.h"
-#include "core/async-table-result-scanner.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-#include "if/Client.pb.h"
-#include "security/user.h"
-#include "serde/server-name.h"
-#include "utils/time-util.h"
-
-using hbase::pb::TableName;
-using hbase::security::User;
-using std::chrono::milliseconds;
-
-namespace hbase {
-
-Table::Table(const TableName &table_name, std::shared_ptr<AsyncConnection> async_connection)
-    : table_name_(std::make_shared<TableName>(table_name)),
-      async_connection_(async_connection),
-      conf_(async_connection->conf()) {
-  async_table_ = std::make_unique<RawAsyncTable>(table_name_, async_connection);
-}
-
-Table::~Table() {}
-
-std::shared_ptr<hbase::Result> Table::Get(const hbase::Get &get) {
-  auto context = async_table_->Get(get);
-  return context.get(operation_timeout());
-}
-
-std::shared_ptr<ResultScanner> Table::Scan(const hbase::Scan &scan) {
-  auto max_cache_size = ResultSize2CacheSize(
-      scan.MaxResultSize() > 0 ? scan.MaxResultSize()
-                               : async_connection_->connection_conf()->scanner_max_result_size());
-  auto scanner = std::make_shared<AsyncTableResultScanner>(max_cache_size);
-  async_table_->Scan(scan, scanner);
-  return scanner;
-}
-
-int64_t Table::ResultSize2CacheSize(int64_t max_results_size) const {
-  // * 2 if possible
-  return max_results_size > (std::numeric_limits<int64_t>::max() / 2) ? max_results_size
-                                                                      : max_results_size * 2;
-}
-
-void Table::Put(const hbase::Put &put) {
-  auto future = async_table_->Put(put);
-  future.get(operation_timeout());
-}
-
-bool Table::CheckAndPut(const std::string &row, const std::string &family,
-                        const std::string &qualifier, const std::string &value,
-                        const hbase::Put &put, const pb::CompareType &compare_op) {
-  auto context = async_table_->CheckAndPut(row, family, qualifier, value, put, compare_op);
-  return context.get(operation_timeout());
-}
-
-bool Table::CheckAndDelete(const std::string &row, const std::string &family,
-                           const std::string &qualifier, const std::string &value,
-                           const hbase::Delete &del, const pb::CompareType &compare_op) {
-  auto context = async_table_->CheckAndDelete(row, family, qualifier, value, del, compare_op);
-  return context.get(operation_timeout());
-}
-
-void Table::Delete(const hbase::Delete &del) {
-  auto future = async_table_->Delete(del);
-  future.get(operation_timeout());
-}
-
-std::shared_ptr<hbase::Result> Table::Increment(const hbase::Increment &increment) {
-  auto context = async_table_->Increment(increment);
-  return context.get(operation_timeout());
-}
-
-std::shared_ptr<hbase::Result> Table::Append(const hbase::Append &append) {
-  auto context = async_table_->Append(append);
-  return context.get(operation_timeout());
-}
-
-milliseconds Table::operation_timeout() const {
-  return TimeUtil::ToMillis(async_connection_->connection_conf()->operation_timeout());
-}
-
-void Table::Close() { async_table_->Close(); }
-
-std::shared_ptr<RegionLocation> Table::GetRegionLocation(const std::string &row) {
-  return async_connection_->region_locator()->LocateRegion(*table_name_, row).get();
-}
-
-std::vector<std::shared_ptr<hbase::Result>> Table::Get(const std::vector<hbase::Get> &gets) {
-  auto tresults = async_table_->Get(gets).get(operation_timeout());
-  std::vector<std::shared_ptr<hbase::Result>> results{};
-  uint32_t num = 0;
-  for (auto tresult : tresults) {
-    if (tresult.hasValue()) {
-      results.push_back(tresult.value());
-    } else if (tresult.hasException()) {
-      LOG(ERROR) << "Caught exception:- " << tresult.exception().what() << " for "
-                 << gets[num++].row();
-      throw tresult.exception();
-    }
-  }
-  return results;
-}
-
-void Table::Put(const std::vector<hbase::Put> &puts) {
-  auto tresults = async_table_->Put(puts).get(operation_timeout());
-  uint32_t num = 0;
-  for (auto tresult : tresults) {
-    if (tresult.hasException()) {
-      LOG(ERROR) << "Caught exception:- " << tresult.exception().what() << " for "
-                 << puts[num++].row();
-      throw tresult.exception();
-    }
-  }
-  return;
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/table.h b/hbase-native-client/core/table.h
deleted file mode 100644
index 6340494..0000000
--- a/hbase-native-client/core/table.h
+++ /dev/null
@@ -1,151 +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.
- *
- */
-
-#pragma once
-
-#include <chrono>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/client.h"
-#include "core/configuration.h"
-#include "core/get.h"
-#include "core/location-cache.h"
-#include "core/put.h"
-#include "core/raw-async-table.h"
-#include "core/result-scanner.h"
-#include "core/result.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-
-class Client;
-
-class Table {
- public:
-  /**
-   * Constructors
-   */
-  Table(const pb::TableName &table_name, std::shared_ptr<AsyncConnection> async_connection);
-  ~Table();
-
-  /**
-   * @brief - Returns a Result object for the constructed Get.
-   * @param - get Get object to perform HBase Get operation.
-   */
-  std::shared_ptr<hbase::Result> Get(const hbase::Get &get);
-
-  std::vector<std::shared_ptr<hbase::Result>> Get(const std::vector<hbase::Get> &gets);
-
-  /**
-   * @brief - Puts some data in the table.
-   * @param - put Put object to perform HBase Put operation.
-   */
-  void Put(const hbase::Put &put);
-
-  /**
-   * Atomically checks if a row/family/qualifier value matches the expected
-   * value. If it does, it adds the put.  If the passed value is null, the check
-   * is for the lack of column (ie: non-existance)
-   *
-   * @param row to check
-   * @param family column family to check
-   * @param qualifier column qualifier to check
-   * @param value the expected value
-   * @param put data to put if check succeeds
-   * @param compare_op comparison operator to use
-   * @throws IOException e
-   * @return true if the new put was executed, false otherwise
-   */
-  bool CheckAndPut(const std::string &row, const std::string &family, const std::string &qualifier,
-                   const std::string &value, const hbase::Put &put,
-                   const pb::CompareType &compare_op = pb::CompareType::EQUAL);
-  /**
-   * @brief - Deletes some data in the table.
-   * @param - del Delete object to perform HBase Delete operation.
-   */
-  void Delete(const hbase::Delete &del);
-
-  /**
-   * Atomically checks if a row/family/qualifier value matches the expected
-   * value. If it does, it adds the delete.  If the passed value is null, the
-   * check is for the lack of column (ie: non-existence)
-   *
-   * The expected value argument of this call is on the left and the current
-   * value of the cell is on the right side of the comparison operator.
-   *
-   * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
-   *
-   * @param row to check
-   * @param family column family to check
-   * @param qualifier column qualifier to check
-   * @param compare_op comparison operator to use
-   * @param value the expected value
-   * @param del data to delete if check succeeds
-   * @return true if the new delete was executed, false otherwise
-   */
-  bool CheckAndDelete(const std::string &row, const std::string &family,
-                      const std::string &qualifier, const std::string &value,
-                      const hbase::Delete &del,
-                      const pb::CompareType &compare_op = pb::CompareType::EQUAL);
-
-  /**
-   * @brief - Increments some data in the table.
-   * @param - increment Increment object to perform HBase Increment operation.
-   */
-  std::shared_ptr<hbase::Result> Increment(const hbase::Increment &increment);
-
-  /**
-   * @brief - Appends some data in the table.
-   * @param - append Append object to perform HBase Append operation.
-   */
-  std::shared_ptr<hbase::Result> Append(const hbase::Append &append);
-
-  std::shared_ptr<ResultScanner> Scan(const hbase::Scan &scan);
-
-  /**
-     * @brief - Multi Puts.
-     * @param - puts vector of hbase::Put.
-     */
-  void Put(const std::vector<hbase::Put> &puts);
-  /**
-   * @brief - Close the client connection.
-   */
-  void Close();
-
-  /**
-   * @brief - Get region location for a row in current table.
-   */
-  std::shared_ptr<RegionLocation> GetRegionLocation(const std::string &row);
-
- private:
-  std::shared_ptr<pb::TableName> table_name_;
-  std::shared_ptr<AsyncConnection> async_connection_;
-  std::shared_ptr<hbase::Configuration> conf_;
-  std::unique_ptr<RawAsyncTable> async_table_;
-
- private:
-  std::chrono::milliseconds operation_timeout() const;
-
-  int64_t ResultSize2CacheSize(int64_t max_results_size) const;
-};
-} /* namespace hbase */


[22/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-rpc-retrying-caller-factory.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-rpc-retrying-caller-factory.h b/hbase-native-client/core/async-rpc-retrying-caller-factory.h
deleted file mode 100644
index 188f469..0000000
--- a/hbase-native-client/core/async-rpc-retrying-caller-factory.h
+++ /dev/null
@@ -1,343 +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.
- *
- */
-#pragma once
-
-#include <folly/Logging.h>
-#include <folly/io/async/EventBase.h>
-#include <chrono>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "connection/rpc-client.h"
-#include "core/async-batch-rpc-retrying-caller.h"
-#include "core/async-rpc-retrying-caller.h"
-#include "core/async-scan-rpc-retrying-caller.h"
-#include "core/raw-scan-result-consumer.h"
-#include "core/region-location.h"
-#include "core/row.h"
-#include "core/scan-result-cache.h"
-#include "core/scan.h"
-
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-class AsyncConnection;
-
-template <typename RESP>
-class SingleRequestCallerBuilder
-    : public std::enable_shared_from_this<SingleRequestCallerBuilder<RESP>> {
- public:
-  explicit SingleRequestCallerBuilder(std::shared_ptr<AsyncConnection> conn,
-                                      std::shared_ptr<folly::HHWheelTimer> retry_timer)
-      : conn_(conn),
-        retry_timer_(retry_timer),
-        table_name_(nullptr),
-        rpc_timeout_nanos_(conn->connection_conf()->rpc_timeout()),
-        pause_(conn->connection_conf()->pause()),
-        operation_timeout_nanos_(conn->connection_conf()->operation_timeout()),
-        max_retries_(conn->connection_conf()->max_retries()),
-        start_log_errors_count_(conn->connection_conf()->start_log_errors_count()),
-        locate_type_(RegionLocateType::kCurrent) {}
-
-  virtual ~SingleRequestCallerBuilder() = default;
-
-  typedef SingleRequestCallerBuilder<RESP> GenericThisType;
-  typedef std::shared_ptr<GenericThisType> SharedThisPtr;
-
-  SharedThisPtr table(std::shared_ptr<pb::TableName> table_name) {
-    table_name_ = table_name;
-    return shared_this();
-  }
-
-  SharedThisPtr rpc_timeout(std::chrono::nanoseconds rpc_timeout_nanos) {
-    rpc_timeout_nanos_ = rpc_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr operation_timeout(std::chrono::nanoseconds operation_timeout_nanos) {
-    operation_timeout_nanos_ = operation_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr pause(std::chrono::nanoseconds pause) {
-    pause_ = pause;
-    return shared_this();
-  }
-
-  SharedThisPtr max_retries(uint32_t max_retries) {
-    max_retries_ = max_retries;
-    return shared_this();
-  }
-
-  SharedThisPtr start_log_errors_count(uint32_t start_log_errors_count) {
-    start_log_errors_count_ = start_log_errors_count;
-    return shared_this();
-  }
-
-  SharedThisPtr row(const std::string& row) {
-    row_ = row;
-    return shared_this();
-  }
-
-  SharedThisPtr locate_type(RegionLocateType locate_type) {
-    locate_type_ = locate_type;
-    return shared_this();
-  }
-
-  SharedThisPtr action(Callable<RESP> callable) {
-    callable_ = callable;
-    return shared_this();
-  }
-
-  folly::Future<RESP> Call() { return Build()->Call(); }
-
-  std::shared_ptr<AsyncSingleRequestRpcRetryingCaller<RESP>> Build() {
-    return std::make_shared<AsyncSingleRequestRpcRetryingCaller<RESP>>(
-        conn_, retry_timer_, table_name_, row_, locate_type_, callable_, pause_, max_retries_,
-        operation_timeout_nanos_, rpc_timeout_nanos_, start_log_errors_count_);
-  }
-
- private:
-  SharedThisPtr shared_this() {
-    return std::enable_shared_from_this<GenericThisType>::shared_from_this();
-  }
-
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<pb::TableName> table_name_;
-  std::chrono::nanoseconds rpc_timeout_nanos_;
-  std::chrono::nanoseconds operation_timeout_nanos_;
-  std::chrono::nanoseconds pause_;
-  uint32_t max_retries_;
-  uint32_t start_log_errors_count_;
-  std::string row_;
-  RegionLocateType locate_type_;
-  Callable<RESP> callable_;
-};  // end of SingleRequestCallerBuilder
-
-template <typename REQ, typename RESP>
-class BatchCallerBuilder : public std::enable_shared_from_this<BatchCallerBuilder<REQ, RESP>> {
- public:
-  explicit BatchCallerBuilder(std::shared_ptr<AsyncConnection> conn,
-                              std::shared_ptr<folly::HHWheelTimer> retry_timer)
-      : conn_(conn), retry_timer_(retry_timer) {}
-
-  virtual ~BatchCallerBuilder() = default;
-
-  typedef std::shared_ptr<BatchCallerBuilder<REQ, RESP>> SharedThisPtr;
-
-  SharedThisPtr table(std::shared_ptr<pb::TableName> table_name) {
-    table_name_ = table_name;
-    return shared_this();
-  }
-
-  SharedThisPtr actions(std::shared_ptr<std::vector<REQ>> actions) {
-    actions_ = actions;
-    return shared_this();
-  }
-
-  SharedThisPtr operation_timeout(std::chrono::nanoseconds operation_timeout_nanos) {
-    operation_timeout_nanos_ = operation_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr rpc_timeout(std::chrono::nanoseconds rpc_timeout_nanos) {
-    rpc_timeout_nanos_ = rpc_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr pause(std::chrono::nanoseconds pause_ns) {
-    pause_ns_ = pause_ns;
-    return shared_this();
-  }
-
-  SharedThisPtr max_attempts(int32_t max_attempts) {
-    max_attempts_ = max_attempts;
-    return shared_this();
-  }
-
-  SharedThisPtr start_log_errors_count(int32_t start_log_errors_count) {
-    start_log_errors_count_ = start_log_errors_count;
-    return shared_this();
-  }
-
-  folly::Future<std::vector<folly::Try<RESP>>> Call() { return Build()->Call(); }
-
-  std::shared_ptr<AsyncBatchRpcRetryingCaller<REQ, RESP>> Build() {
-    return std::make_shared<AsyncBatchRpcRetryingCaller<REQ, RESP>>(
-        conn_, retry_timer_, table_name_, *actions_, pause_ns_, max_attempts_,
-        operation_timeout_nanos_, rpc_timeout_nanos_, start_log_errors_count_);
-  }
-
- private:
-  SharedThisPtr shared_this() {
-    return std::enable_shared_from_this<BatchCallerBuilder>::shared_from_this();
-  }
-
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<hbase::pb::TableName> table_name_ = nullptr;
-  std::shared_ptr<std::vector<REQ>> actions_ = nullptr;
-  std::chrono::nanoseconds pause_ns_;
-  int32_t max_attempts_ = 0;
-  std::chrono::nanoseconds operation_timeout_nanos_;
-  std::chrono::nanoseconds rpc_timeout_nanos_;
-  int32_t start_log_errors_count_ = 0;
-};
-
-class ScanCallerBuilder : public std::enable_shared_from_this<ScanCallerBuilder> {
- public:
-  explicit ScanCallerBuilder(std::shared_ptr<AsyncConnection> conn,
-                             std::shared_ptr<folly::HHWheelTimer> retry_timer)
-      : conn_(conn),
-        retry_timer_(retry_timer),
-        rpc_timeout_nanos_(conn->connection_conf()->rpc_timeout()),
-        pause_(conn->connection_conf()->pause()),
-        scan_timeout_nanos_(conn->connection_conf()->scan_timeout()),
-        max_retries_(conn->connection_conf()->max_retries()),
-        start_log_errors_count_(conn->connection_conf()->start_log_errors_count()),
-        scanner_id_(-1) {}
-
-  virtual ~ScanCallerBuilder() = default;
-
-  typedef ScanCallerBuilder GenericThisType;
-  typedef std::shared_ptr<ScanCallerBuilder> SharedThisPtr;
-
-  SharedThisPtr rpc_client(std::shared_ptr<hbase::RpcClient> rpc_client) {
-    rpc_client_ = rpc_client;
-    return shared_this();
-  }
-
-  SharedThisPtr rpc_timeout(nanoseconds rpc_timeout_nanos) {
-    rpc_timeout_nanos_ = rpc_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr scan_timeout(nanoseconds scan_timeout_nanos) {
-    scan_timeout_nanos_ = scan_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr scanner_lease_timeout(nanoseconds scanner_lease_timeout_nanos) {
-    scanner_lease_timeout_nanos_ = scanner_lease_timeout_nanos;
-    return shared_this();
-  }
-
-  SharedThisPtr pause(nanoseconds pause) {
-    pause_ = pause;
-    return shared_this();
-  }
-
-  SharedThisPtr max_retries(uint32_t max_retries) {
-    max_retries_ = max_retries;
-    return shared_this();
-  }
-
-  SharedThisPtr start_log_errors_count(uint32_t start_log_errors_count) {
-    start_log_errors_count_ = start_log_errors_count;
-    return shared_this();
-  }
-
-  SharedThisPtr region_location(std::shared_ptr<RegionLocation> region_location) {
-    region_location_ = region_location;
-    return shared_this();
-  }
-
-  SharedThisPtr scanner_id(int64_t scanner_id) {
-    scanner_id_ = scanner_id;
-    return shared_this();
-  }
-
-  SharedThisPtr scan(std::shared_ptr<Scan> scan) {
-    scan_ = scan;
-    return shared_this();
-  }
-
-  SharedThisPtr results_cache(std::shared_ptr<ScanResultCache> results_cache) {
-    results_cache_ = results_cache;
-    return shared_this();
-  }
-
-  SharedThisPtr consumer(std::shared_ptr<RawScanResultConsumer> consumer) {
-    consumer_ = consumer;
-    return shared_this();
-  }
-
-  std::shared_ptr<AsyncScanRpcRetryingCaller> Build() {
-    return std::make_shared<AsyncScanRpcRetryingCaller>(
-        conn_, retry_timer_, rpc_client_, scan_, scanner_id_, results_cache_, consumer_,
-        region_location_, scanner_lease_timeout_nanos_, pause_, max_retries_, scan_timeout_nanos_,
-        rpc_timeout_nanos_, start_log_errors_count_);
-  }
-
- private:
-  SharedThisPtr shared_this() {
-    return std::enable_shared_from_this<GenericThisType>::shared_from_this();
-  }
-
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<hbase::RpcClient> rpc_client_;
-  std::shared_ptr<Scan> scan_;
-  nanoseconds rpc_timeout_nanos_;
-  nanoseconds scan_timeout_nanos_;
-  nanoseconds scanner_lease_timeout_nanos_;
-  nanoseconds pause_;
-  uint32_t max_retries_;
-  uint32_t start_log_errors_count_;
-  std::shared_ptr<RegionLocation> region_location_;
-  int64_t scanner_id_;
-  std::shared_ptr<RawScanResultConsumer> consumer_;
-  std::shared_ptr<ScanResultCache> results_cache_;
-};  // end of ScanCallerBuilder
-
-class AsyncRpcRetryingCallerFactory {
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-
- public:
-  explicit AsyncRpcRetryingCallerFactory(std::shared_ptr<AsyncConnection> conn,
-                                         std::shared_ptr<folly::HHWheelTimer> retry_timer)
-      : conn_(conn), retry_timer_(retry_timer) {}
-
-  virtual ~AsyncRpcRetryingCallerFactory() = default;
-
-  template <typename RESP>
-  std::shared_ptr<SingleRequestCallerBuilder<RESP>> Single() {
-    return std::make_shared<SingleRequestCallerBuilder<RESP>>(conn_, retry_timer_);
-  }
-
-  template <typename REQ, typename RESP>
-  std::shared_ptr<BatchCallerBuilder<REQ, RESP>> Batch() {
-    return std::make_shared<BatchCallerBuilder<REQ, RESP>>(conn_, retry_timer_);
-  }
-
-  std::shared_ptr<ScanCallerBuilder> Scan() {
-    return std::make_shared<ScanCallerBuilder>(conn_, retry_timer_);
-  }
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-rpc-retrying-caller.cc b/hbase-native-client/core/async-rpc-retrying-caller.cc
deleted file mode 100644
index 8e60991..0000000
--- a/hbase-native-client/core/async-rpc-retrying-caller.cc
+++ /dev/null
@@ -1,230 +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/async-rpc-retrying-caller.h"
-
-#include <folly/Conv.h>
-#include <folly/ExceptionWrapper.h>
-#include <folly/Format.h>
-#include <folly/Logging.h>
-#include <folly/Unit.h>
-
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/region-location.h"
-#include "core/result.h"
-#include "exceptions/exception.h"
-#include "if/HBase.pb.h"
-#include "utils/connection-util.h"
-#include "utils/sys-util.h"
-#include "utils/time-util.h"
-
-using folly::exception_wrapper;
-
-namespace hbase {
-
-template <typename RESP>
-AsyncSingleRequestRpcRetryingCaller<RESP>::AsyncSingleRequestRpcRetryingCaller(
-    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
-    std::shared_ptr<hbase::pb::TableName> table_name, const std::string& row,
-    RegionLocateType locate_type, Callable<RESP> callable, std::chrono::nanoseconds pause,
-    uint32_t max_retries, std::chrono::nanoseconds operation_timeout_nanos,
-    std::chrono::nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
-    : conn_(conn),
-      retry_timer_(retry_timer),
-      table_name_(table_name),
-      row_(row),
-      locate_type_(locate_type),
-      callable_(callable),
-      pause_(pause),
-      max_retries_(max_retries),
-      operation_timeout_nanos_(operation_timeout_nanos),
-      rpc_timeout_nanos_(rpc_timeout_nanos),
-      start_log_errors_count_(start_log_errors_count),
-      promise_(std::make_shared<folly::Promise<RESP>>()),
-      tries_(1) {
-  controller_ = conn_->CreateRpcController();
-  start_ns_ = TimeUtil::GetNowNanos();
-  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
-  exceptions_ = std::make_shared<std::vector<ThrowableWithExtraContext>>();
-}
-
-template <typename RESP>
-AsyncSingleRequestRpcRetryingCaller<RESP>::~AsyncSingleRequestRpcRetryingCaller() {}
-
-template <typename RESP>
-folly::Future<RESP> AsyncSingleRequestRpcRetryingCaller<RESP>::Call() {
-  auto f = promise_->getFuture();
-  LocateThenCall();
-  return f;
-}
-
-template <typename RESP>
-void AsyncSingleRequestRpcRetryingCaller<RESP>::LocateThenCall() {
-  int64_t locate_timeout_ns;
-  if (operation_timeout_nanos_.count() > 0) {
-    locate_timeout_ns = RemainingTimeNs();
-    if (locate_timeout_ns <= 0) {
-      CompleteExceptionally();
-      return;
-    }
-  } else {
-    locate_timeout_ns = -1L;
-  }
-
-  conn_->region_locator()
-      ->LocateRegion(*table_name_, row_, locate_type_, locate_timeout_ns)
-      .then([this](std::shared_ptr<RegionLocation> loc) { Call(*loc); })
-      .onError([this](const exception_wrapper& e) {
-        OnError(e,
-                [this, e]() -> std::string {
-                  return "Locate '" + row_ + "' in " + table_name_->namespace_() + "::" +
-                         table_name_->qualifier() + " failed with e.what()=" +
-                         e.what().toStdString() + ", tries = " + std::to_string(tries_) +
-                         ", maxAttempts = " + std::to_string(max_attempts_) + ", timeout = " +
-                         TimeUtil::ToMillisStr(operation_timeout_nanos_) + " ms, time elapsed = " +
-                         TimeUtil::ElapsedMillisStr(this->start_ns_) + " ms";
-                },
-                [](const exception_wrapper& error) {});
-      });
-}
-
-template <typename RESP>
-void AsyncSingleRequestRpcRetryingCaller<RESP>::OnError(
-    const exception_wrapper& error, Supplier<std::string> err_msg,
-    Consumer<exception_wrapper> update_cached_location) {
-  ThrowableWithExtraContext twec(error, TimeUtil::GetNowNanos());
-  exceptions_->push_back(twec);
-  if (!ExceptionUtil::ShouldRetry(error) || tries_ >= max_retries_) {
-    CompleteExceptionally();
-    return;
-  }
-
-  if (tries_ > start_log_errors_count_) {
-    LOG(WARNING) << err_msg();
-  } else {
-    VLOG(1) << err_msg();
-  }
-
-  int64_t delay_ns;
-  if (operation_timeout_nanos_.count() > 0) {
-    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
-    if (max_delay_ns <= 0) {
-      CompleteExceptionally();
-      return;
-    }
-    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1));
-  } else {
-    delay_ns = ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1);
-  }
-  update_cached_location(error);
-  tries_++;
-
-  /*
-   * The HHWheelTimer::scheduleTimeout() fails with an assertion from
-   * EventBase::isInEventBaseThread() if we execute the schedule in a random thread, or one of
-   * the IOThreadPool threads (with num threads > 1). I think there is a bug there in using retry
-   * timer from IOThreadPool threads. It only works when executed from a single-thread pool
-   * (retry_executor() is). However, the scheduled "work" which is the LocateThenCall() should
-   * still happen in a thread pool, that is why we are submitting the work to the CPUThreadPool.
-   * IOThreadPool cannot be used without fixing the blocking call that we do at TCP connection
-   * establishment time (see ConnectionFactory::Connect()), otherwise, the IOThreadPool thread
-   * just hangs because it deadlocks itself.
-   */
-  conn_->retry_executor()->add([=]() {
-    retry_timer_->scheduleTimeoutFn(
-        [=]() { conn_->cpu_executor()->add([&]() { LocateThenCall(); }); },
-        std::chrono::milliseconds(TimeUtil::ToMillis(delay_ns)));
-  });
-}
-
-template <typename RESP>
-void AsyncSingleRequestRpcRetryingCaller<RESP>::Call(const RegionLocation& loc) {
-  int64_t call_timeout_ns;
-  if (operation_timeout_nanos_.count() > 0) {
-    call_timeout_ns = this->RemainingTimeNs();
-    if (call_timeout_ns <= 0) {
-      this->CompleteExceptionally();
-      return;
-    }
-    call_timeout_ns = std::min(call_timeout_ns, rpc_timeout_nanos_.count());
-  } else {
-    call_timeout_ns = rpc_timeout_nanos_.count();
-  }
-
-  std::shared_ptr<RpcClient> rpc_client;
-
-  rpc_client = conn_->rpc_client();
-
-  ResetController(controller_, call_timeout_ns);
-
-  // TODO: RegionLocation should propagate through these method chains as a shared_ptr.
-  // Otherwise, it may get deleted underneat us. We are just copying for now.
-  auto loc_ptr = std::make_shared<RegionLocation>(loc);
-  callable_(controller_, loc_ptr, rpc_client)
-      .then([loc_ptr, this](const RESP& resp) { this->promise_->setValue(std::move(resp)); })
-      .onError([&, loc_ptr, this](const exception_wrapper& e) {
-        OnError(
-            e,
-            [&, this, e]() -> std::string {
-              return "Call to " + folly::sformat("{0}:{1}", loc_ptr->server_name().host_name(),
-                                                 loc_ptr->server_name().port()) +
-                     " for '" + row_ + "' in " + loc_ptr->DebugString() + " of " +
-                     table_name_->namespace_() + "::" + table_name_->qualifier() +
-                     " failed with e.what()=" + e.what().toStdString() + ", tries = " +
-                     std::to_string(tries_) + ", maxAttempts = " + std::to_string(max_attempts_) +
-                     ", timeout = " + TimeUtil::ToMillisStr(this->operation_timeout_nanos_) +
-                     " ms, time elapsed = " + TimeUtil::ElapsedMillisStr(this->start_ns_) + " ms";
-            },
-            [&, this](const exception_wrapper& error) {
-              conn_->region_locator()->UpdateCachedLocation(*loc_ptr, error);
-            });
-      });
-}
-
-template <typename RESP>
-void AsyncSingleRequestRpcRetryingCaller<RESP>::CompleteExceptionally() {
-  this->promise_->setException(RetriesExhaustedException(tries_ - 1, exceptions_));
-}
-
-template <typename RESP>
-int64_t AsyncSingleRequestRpcRetryingCaller<RESP>::RemainingTimeNs() {
-  return operation_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
-}
-
-template <typename RESP>
-void AsyncSingleRequestRpcRetryingCaller<RESP>::ResetController(
-    std::shared_ptr<HBaseRpcController> controller, const int64_t& timeout_ns) {
-  controller->Reset();
-  if (timeout_ns >= 0) {
-    controller->set_call_timeout(std::chrono::milliseconds(
-        std::min(static_cast<int64_t>(INT_MAX), TimeUtil::ToMillis(timeout_ns))));
-  }
-}
-
-// explicit instantiations for the linker. Otherwise, you have to #include the .cc file for the
-// templetized
-// class definitions.
-class OpenScannerResponse;
-template class AsyncSingleRequestRpcRetryingCaller<std::shared_ptr<hbase::Result>>;
-template class AsyncSingleRequestRpcRetryingCaller<folly::Unit>;
-template class AsyncSingleRequestRpcRetryingCaller<std::shared_ptr<OpenScannerResponse>>;
-template class AsyncSingleRequestRpcRetryingCaller<bool>;
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-rpc-retrying-caller.h b/hbase-native-client/core/async-rpc-retrying-caller.h
deleted file mode 100644
index c7e28d0..0000000
--- a/hbase-native-client/core/async-rpc-retrying-caller.h
+++ /dev/null
@@ -1,115 +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.
- *
- */
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/futures/Future.h>
-#include <folly/io/async/EventBase.h>
-#include <folly/io/async/HHWheelTimer.h>
-
-#include <algorithm>
-#include <chrono>
-#include <functional>
-#include <memory>
-#include <string>
-#include <type_traits>
-#include <utility>
-#include <vector>
-#include "core/async-connection.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/region-location.h"
-#include "exceptions/exception.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-template <typename T>
-using Supplier = std::function<T()>;
-
-template <typename T>
-using Consumer = std::function<void(T)>;
-
-template <typename R, typename S, typename... I>
-using ReqConverter = std::function<R(const S&, const I&...)>;
-
-template <typename R, typename S>
-using RespConverter = std::function<R(const S&)>;
-
-template <typename RESP>
-using RpcCallback = std::function<void(const RESP&)>;
-
-template <typename REQ, typename RESP>
-using RpcCall = std::function<folly::Future<std::unique_ptr<RESP>>(
-    std::shared_ptr<RpcClient>, std::shared_ptr<RegionLocation>,
-    std::shared_ptr<HBaseRpcController>, std::unique_ptr<REQ>)>;
-
-template <typename RESP>
-using Callable =
-    std::function<folly::Future<RESP>(std::shared_ptr<HBaseRpcController>,
-                                      std::shared_ptr<RegionLocation>, std::shared_ptr<RpcClient>)>;
-
-template <typename RESP>
-class AsyncSingleRequestRpcRetryingCaller {
- public:
-  AsyncSingleRequestRpcRetryingCaller(
-      std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
-      std::shared_ptr<hbase::pb::TableName> table_name, const std::string& row,
-      RegionLocateType locate_type, Callable<RESP> callable, std::chrono::nanoseconds pause,
-      uint32_t max_retries, std::chrono::nanoseconds operation_timeout_nanos,
-      std::chrono::nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
-
-  virtual ~AsyncSingleRequestRpcRetryingCaller();
-
-  folly::Future<RESP> Call();
-
- private:
-  void LocateThenCall();
-
-  void OnError(const folly::exception_wrapper& error, Supplier<std::string> err_msg,
-               Consumer<folly::exception_wrapper> update_cached_location);
-
-  void Call(const RegionLocation& loc);
-
-  void CompleteExceptionally();
-
-  int64_t RemainingTimeNs();
-
-  static void ResetController(std::shared_ptr<HBaseRpcController> controller,
-                              const int64_t& timeout_ns);
-
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<hbase::pb::TableName> table_name_;
-  std::string row_;
-  RegionLocateType locate_type_;
-  Callable<RESP> callable_;
-  std::chrono::nanoseconds pause_;
-  uint32_t max_retries_;
-  std::chrono::nanoseconds operation_timeout_nanos_;
-  std::chrono::nanoseconds rpc_timeout_nanos_;
-  uint32_t start_log_errors_count_;
-  std::shared_ptr<folly::Promise<RESP>> promise_;
-  std::shared_ptr<HBaseRpcController> controller_;
-  uint64_t start_ns_;
-  uint32_t tries_;
-  std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions_;
-  uint32_t max_attempts_;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-rpc-retrying-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-rpc-retrying-test.cc b/hbase-native-client/core/async-rpc-retrying-test.cc
deleted file mode 100644
index 2eb82a9..0000000
--- a/hbase-native-client/core/async-rpc-retrying-test.cc
+++ /dev/null
@@ -1,420 +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 <folly/Logging.h>
-#include <folly/Memory.h>
-#include <folly/futures/Future.h>
-#include <folly/io/async/EventBase.h>
-#include <folly/io/async/ScopedEventBaseThread.h>
-#include <gmock/gmock.h>
-#include <google/protobuf/stubs/callback.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-
-#include <chrono>
-#include <functional>
-#include <string>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/async-rpc-retrying-caller-factory.h"
-#include "core/async-rpc-retrying-caller.h"
-#include "core/client.h"
-#include "core/connection-configuration.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/keyvalue-codec.h"
-#include "core/region-location.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-#include "core/result.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-#include "test-util/test-util.h"
-#include "utils/time-util.h"
-
-using hbase::AsyncRpcRetryingCallerFactory;
-using hbase::AsyncConnection;
-using hbase::AsyncRegionLocator;
-using hbase::ConnectionConfiguration;
-using hbase::Configuration;
-using hbase::HBaseRpcController;
-using hbase::RegionLocation;
-using hbase::RegionLocateType;
-using hbase::RpcClient;
-using hbase::RequestConverter;
-using hbase::ResponseConverter;
-using hbase::ReqConverter;
-using hbase::RespConverter;
-using hbase::Put;
-using hbase::TimeUtil;
-using hbase::Client;
-using hbase::security::User;
-
-using ::testing::Return;
-using ::testing::_;
-using std::chrono::nanoseconds;
-using std::chrono::milliseconds;
-
-using namespace hbase;
-
-using folly::exception_wrapper;
-
-class AsyncRpcRetryTest : public ::testing::Test {
- public:
-  static std::unique_ptr<hbase::TestUtil> test_util;
-
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util = std::make_unique<hbase::TestUtil>();
-    test_util->StartMiniCluster(2);
-  }
-};
-std::unique_ptr<hbase::TestUtil> AsyncRpcRetryTest::test_util = nullptr;
-
-class AsyncRegionLocatorBase : public AsyncRegionLocator {
- public:
-  AsyncRegionLocatorBase() {}
-  explicit AsyncRegionLocatorBase(std::shared_ptr<RegionLocation> region_location)
-      : region_location_(region_location) {}
-  virtual ~AsyncRegionLocatorBase() = default;
-
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(const hbase::pb::TableName &,
-                                                                     const std::string &,
-                                                                     const RegionLocateType,
-                                                                     const int64_t) override {
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    promise.setValue(region_location_);
-    return promise.getFuture();
-  }
-
-  virtual void set_region_location(std::shared_ptr<RegionLocation> region_location) {
-    region_location_ = region_location;
-  }
-
-  void UpdateCachedLocation(const RegionLocation &, const folly::exception_wrapper &) override {}
-
- protected:
-  std::shared_ptr<RegionLocation> region_location_;
-};
-
-class MockAsyncRegionLocator : public AsyncRegionLocatorBase {
- public:
-  MockAsyncRegionLocator() : AsyncRegionLocatorBase() {}
-  explicit MockAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockAsyncRegionLocator() {}
-};
-
-class MockWrongRegionAsyncRegionLocator : public AsyncRegionLocatorBase {
- private:
-  uint32_t tries_ = 0;
-  uint32_t num_fails_ = 0;
-
- public:
-  explicit MockWrongRegionAsyncRegionLocator(uint32_t num_fails)
-      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
-  explicit MockWrongRegionAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockWrongRegionAsyncRegionLocator() {}
-
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) override {
-    // Fail for num_fails_ times, then delegate to the super class which will give the correct
-    // region location.
-    if (tries_++ > num_fails_) {
-      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
-    }
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    /* set random region name, simulating invalid region */
-    auto result = std::make_shared<RegionLocation>(
-        "whatever-region-name", region_location_->region_info(), region_location_->server_name());
-    promise.setValue(result);
-    return promise.getFuture();
-  }
-};
-
-class MockFailingAsyncRegionLocator : public AsyncRegionLocatorBase {
- private:
-  uint32_t tries_ = 0;
-  uint32_t num_fails_ = 0;
-
- public:
-  explicit MockFailingAsyncRegionLocator(uint32_t num_fails)
-      : AsyncRegionLocatorBase(), num_fails_(num_fails) {}
-  explicit MockFailingAsyncRegionLocator(std::shared_ptr<RegionLocation> region_location)
-      : AsyncRegionLocatorBase(region_location) {}
-  virtual ~MockFailingAsyncRegionLocator() {}
-  folly::Future<std::shared_ptr<hbase::RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) override {
-    // Fail for num_fails_ times, then delegate to the super class which will give the correct
-    // region location.
-    if (tries_++ > num_fails_) {
-      return AsyncRegionLocatorBase::LocateRegion(tn, row, locate_type, locate_ns);
-    }
-    folly::Promise<std::shared_ptr<RegionLocation>> promise;
-    promise.setException(std::runtime_error{"Failed to look up region location"});
-    return promise.getFuture();
-  }
-};
-
-class MockAsyncConnection : public AsyncConnection,
-                            public std::enable_shared_from_this<MockAsyncConnection> {
- public:
-  MockAsyncConnection(std::shared_ptr<ConnectionConfiguration> conn_conf,
-                      std::shared_ptr<folly::HHWheelTimer> retry_timer,
-                      std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                      std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                      std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor,
-                      std::shared_ptr<RpcClient> rpc_client,
-                      std::shared_ptr<AsyncRegionLocator> region_locator)
-      : conn_conf_(conn_conf),
-        retry_timer_(retry_timer),
-        cpu_executor_(cpu_executor),
-        io_executor_(io_executor),
-        retry_executor_(retry_executor),
-        rpc_client_(rpc_client),
-        region_locator_(region_locator) {}
-  ~MockAsyncConnection() {}
-  void Init() {
-    caller_factory_ =
-        std::make_shared<AsyncRpcRetryingCallerFactory>(shared_from_this(), retry_timer_);
-  }
-
-  std::shared_ptr<Configuration> conf() override { return nullptr; }
-  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return conn_conf_; }
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
-    return caller_factory_;
-  }
-  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
-  std::shared_ptr<AsyncRegionLocator> region_locator() override { return region_locator_; }
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
-    return retry_executor_;
-  }
-
-  void Close() override {}
-  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
-    return std::make_shared<HBaseRpcController>();
-  }
-
- private:
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<ConnectionConfiguration> conn_conf_;
-  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
-  std::shared_ptr<RpcClient> rpc_client_;
-  std::shared_ptr<AsyncRegionLocator> region_locator_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
-};
-
-template <typename CONN>
-class MockRawAsyncTableImpl {
- public:
-  explicit MockRawAsyncTableImpl(std::shared_ptr<CONN> conn) : conn_(conn) {}
-  virtual ~MockRawAsyncTableImpl() = default;
-
-  /* implement this in real RawAsyncTableImpl. */
-
-  /* in real RawAsyncTableImpl, this should be private. */
-  folly::Future<std::shared_ptr<hbase::Result>> GetCall(
-      std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
-      std::shared_ptr<RegionLocation> loc, const hbase::Get &get) {
-    hbase::RpcCall<hbase::Request, hbase::Response> rpc_call = [](
-        std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<RegionLocation> loc,
-        std::shared_ptr<HBaseRpcController> controller,
-        std::unique_ptr<hbase::Request> preq) -> folly::Future<std::unique_ptr<hbase::Response>> {
-      VLOG(1) << "entering MockRawAsyncTableImpl#GetCall, calling AsyncCall, loc:"
-              << loc->DebugString();
-      return rpc_client->AsyncCall(loc->server_name().host_name(), loc->server_name().port(),
-                                   std::move(preq), User::defaultUser(), "ClientService");
-    };
-
-    return Call<hbase::Get, hbase::Request, hbase::Response, std::shared_ptr<hbase::Result>>(
-        rpc_client, controller, loc, get, &hbase::RequestConverter::ToGetRequest, rpc_call,
-        &hbase::ResponseConverter::FromGetResponse);
-  }
-
-  /* in real RawAsyncTableImpl, this should be private. */
-  template <typename REQ, typename PREQ, typename PRESP, typename RESP>
-  folly::Future<RESP> Call(std::shared_ptr<hbase::RpcClient> rpc_client,
-                           std::shared_ptr<HBaseRpcController> controller,
-                           std::shared_ptr<RegionLocation> loc, const REQ &req,
-                           ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
-                           hbase::RpcCall<PREQ, PRESP> rpc_call,
-                           RespConverter<RESP, PRESP> resp_converter) {
-    promise_ = std::make_shared<folly::Promise<std::shared_ptr<hbase::Result>>>();
-    auto f = promise_->getFuture();
-    VLOG(1) << "calling rpc_call";
-    rpc_call(rpc_client, loc, controller, std::move(req_converter(req, loc->region_name())))
-        .then([&, this, resp_converter](std::unique_ptr<PRESP> presp) {
-          VLOG(1) << "MockRawAsyncTableImpl#call succeded: ";
-          RESP result = resp_converter(*presp);
-          promise_->setValue(result);
-        })
-        .onError([this](const exception_wrapper &e) {
-          VLOG(1) << "entering MockRawAsyncTableImpl#call, exception: " << e.what();
-          VLOG(1) << "entering MockRawAsyncTableImpl#call, error typeinfo: " << typeid(e).name();
-          promise_->setException(e);
-        });
-    return f;
-  }
-
- private:
-  std::shared_ptr<CONN> conn_;
-  std::shared_ptr<folly::Promise<std::shared_ptr<hbase::Result>>> promise_;
-};
-
-void runTest(std::shared_ptr<AsyncRegionLocatorBase> region_locator, std::string tableName,
-             uint32_t operation_timeout_millis = 1200000) {
-  AsyncRpcRetryTest::test_util->CreateTable(tableName, "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(tableName);
-  auto row = "test2";
-
-  // Get to be performed on above HBase Table
-  hbase::Get get(row);
-
-  // Create a client
-  Client client(*(AsyncRpcRetryTest::test_util->conf()));
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-
-  table->Put(Put{"test2"}.AddColumn("d", "2", "value2"));
-  table->Put(Put{"test2"}.AddColumn("d", "extra", "value for extra"));
-
-  /* init region location and rpc channel */
-  auto region_location = table->GetRegionLocation(row);
-
-  // auto io_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(4);
-  auto cpu_executor_ = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
-  auto io_executor_ = client.async_connection()->io_executor();
-  auto retry_executor_ = std::make_shared<wangle::IOThreadPoolExecutor>(1);
-  auto codec = std::make_shared<hbase::KeyValueCodec>();
-  auto rpc_client = std::make_shared<RpcClient>(io_executor_, cpu_executor_, codec,
-                                                AsyncRpcRetryTest::test_util->conf());
-  // auto retry_event_base_ = std::make_shared<folly::ScopedEventBaseThread>(true);
-  std::shared_ptr<folly::HHWheelTimer> retry_timer =
-      folly::HHWheelTimer::newTimer(retry_executor_->getEventBase());
-
-  /* init connection configuration */
-  auto connection_conf = std::make_shared<ConnectionConfiguration>(
-      TimeUtil::SecondsToNanos(20),                       // connect_timeout
-      TimeUtil::MillisToNanos(operation_timeout_millis),  // operation_timeout
-      TimeUtil::SecondsToNanos(60),                       // rpc_timeout
-      TimeUtil::MillisToNanos(100),                       // pause
-      5,                                                  // max retries
-      9);                                                 // start log errors count
-
-  /* set region locator */
-  region_locator->set_region_location(region_location);
-
-  /* init hbase client connection */
-  auto conn = std::make_shared<MockAsyncConnection>(connection_conf, retry_timer, cpu_executor_,
-                                                    io_executor_, retry_executor_, rpc_client,
-                                                    region_locator);
-  conn->Init();
-
-  /* init retry caller factory */
-  auto tableImpl = std::make_shared<MockRawAsyncTableImpl<MockAsyncConnection>>(conn);
-
-  /* init request caller builder */
-  auto builder = conn->caller_factory()->Single<std::shared_ptr<hbase::Result>>();
-
-  /* call with retry to get result */
-
-  auto async_caller =
-      builder->table(std::make_shared<hbase::pb::TableName>(tn))
-          ->row(row)
-          ->rpc_timeout(conn->connection_conf()->read_rpc_timeout())
-          ->operation_timeout(conn->connection_conf()->operation_timeout())
-          ->action([=, &get](std::shared_ptr<hbase::HBaseRpcController> controller,
-                             std::shared_ptr<hbase::RegionLocation> loc,
-                             std::shared_ptr<hbase::RpcClient> rpc_client)
-                       -> folly::Future<std::shared_ptr<hbase::Result>> {
-                         return tableImpl->GetCall(rpc_client, controller, loc, get);
-                       })
-          ->Build();
-
-  auto promise = std::make_shared<folly::Promise<std::shared_ptr<hbase::Result>>>();
-
-  auto result = async_caller->Call().get(milliseconds(500000));
-
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ("test2", result->Row());
-  EXPECT_EQ("value2", *(result->Value("d", "2")));
-  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
-
-  retry_timer->destroy();
-  table->Close();
-  client.Close();
-  retry_executor_->stop();
-}
-
-// Test successful case
-TEST_F(AsyncRpcRetryTest, TestGetBasic) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockAsyncRegionLocator>());
-  runTest(region_locator, "table1");
-}
-
-// Tests the RPC failing 3 times, then succeeding
-TEST_F(AsyncRpcRetryTest, TestHandleException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(3));
-  runTest(region_locator, "table2");
-}
-
-// Tests the RPC failing 5 times, throwing an exception
-TEST_F(AsyncRpcRetryTest, TestFailWithException) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockWrongRegionAsyncRegionLocator>(5));
-  EXPECT_ANY_THROW(runTest(region_locator, "table3"));
-}
-
-// Tests the region location lookup failing 3 times, then succeeding
-TEST_F(AsyncRpcRetryTest, TestHandleExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(3));
-  runTest(region_locator, "table4");
-}
-
-// Tests the region location lookup failing 5 times, throwing an exception
-TEST_F(AsyncRpcRetryTest, TestFailWithExceptionFromRegionLocationLookup) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(5));
-  EXPECT_ANY_THROW(runTest(region_locator, "table5"));
-}
-
-// Tests hitting operation timeout, thus not retrying anymore
-TEST_F(AsyncRpcRetryTest, TestFailWithOperationTimeout) {
-  std::shared_ptr<AsyncRegionLocatorBase> region_locator(
-      std::make_shared<MockFailingAsyncRegionLocator>(3));
-  EXPECT_ANY_THROW(runTest(region_locator, "table6", 200));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-scan-rpc-retrying-caller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-scan-rpc-retrying-caller.cc b/hbase-native-client/core/async-scan-rpc-retrying-caller.cc
deleted file mode 100644
index a1e8362..0000000
--- a/hbase-native-client/core/async-scan-rpc-retrying-caller.cc
+++ /dev/null
@@ -1,448 +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/async-scan-rpc-retrying-caller.h"
-
-namespace hbase {
-
-ScanResumerImpl::ScanResumerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller)
-    : caller_(caller), mutex_() {}
-
-void ScanResumerImpl::Resume() {
-  // just used to fix findbugs warnings. In fact, if resume is called before prepare, then we
-  // just return at the first if condition without loading the resp and numValidResuls field. If
-  // resume is called after suspend, then it is also safe to just reference resp and
-  // numValidResults after the synchronized block as no one will change it anymore.
-  std::shared_ptr<pb::ScanResponse> local_resp;
-  int64_t local_num_complete_rows;
-
-  {
-    std::unique_lock<std::mutex> mlock{mutex_};
-    if (state_ == ScanResumerState::kInitialized) {
-      // user calls this method before we call prepare, so just set the state to
-      // RESUMED, the implementation will just go on.
-      state_ = ScanResumerState::kResumed;
-      return;
-    }
-    if (state_ == ScanResumerState::kResumed) {
-      // already resumed, give up.
-      return;
-    }
-    state_ = ScanResumerState::kResumed;
-    local_resp = resp_;
-    local_num_complete_rows = num_complete_rows_;
-  }
-
-  caller_->CompleteOrNext(local_resp);
-}
-
-bool ScanResumerImpl::Prepare(std::shared_ptr<pb::ScanResponse> resp, int num_complete_rows) {
-  std::unique_lock<std::mutex> mlock(mutex_);
-  if (state_ == ScanResumerState::kResumed) {
-    // user calls resume before we actually suspend the scan, just continue;
-    return false;
-  }
-  state_ = ScanResumerState::kSuspended;
-  resp_ = resp;
-  num_complete_rows_ = num_complete_rows;
-
-  return true;
-}
-
-ScanControllerImpl::ScanControllerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller)
-    : caller_(caller) {}
-
-std::shared_ptr<ScanResumer> ScanControllerImpl::Suspend() {
-  PreCheck();
-  state_ = ScanControllerState::kSuspended;
-  resumer_ = std::make_shared<ScanResumerImpl>(caller_);
-  return resumer_;
-}
-
-void ScanControllerImpl::Terminate() {
-  PreCheck();
-  state_ = ScanControllerState::kTerminated;
-}
-
-// return the current state, and set the state to DESTROYED.
-ScanControllerState ScanControllerImpl::Destroy() {
-  ScanControllerState state = state_;
-  state_ = ScanControllerState::kDestroyed;
-  return state;
-}
-
-void ScanControllerImpl::PreCheck() {
-  CHECK(std::this_thread::get_id() == caller_thread_id_)
-      << "The current thread is" << std::this_thread::get_id() << ", expected thread is "
-      << caller_thread_id_ << ", you should not call this method outside OnNext or OnHeartbeat";
-
-  CHECK(state_ == ScanControllerState::kInitialized) << "Invalid Stopper state "
-                                                     << DebugString(state_);
-}
-
-std::string ScanControllerImpl::DebugString(ScanControllerState state) {
-  switch (state) {
-    case ScanControllerState::kInitialized:
-      return "kInitialized";
-    case ScanControllerState::kSuspended:
-      return "kSuspended";
-    case ScanControllerState::kTerminated:
-      return "kTerminated";
-    case ScanControllerState::kDestroyed:
-      return "kDestroyed";
-    default:
-      return "UNKNOWN";
-  }
-}
-
-std::string ScanControllerImpl::DebugString(ScanResumerState state) {
-  switch (state) {
-    case ScanResumerState::kInitialized:
-      return "kInitialized";
-    case ScanResumerState::kSuspended:
-      return "kSuspended";
-    case ScanResumerState::kResumed:
-      return "kResumed";
-    default:
-      return "UNKNOWN";
-  }
-}
-
-AsyncScanRpcRetryingCaller::AsyncScanRpcRetryingCaller(
-    std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
-    std::shared_ptr<hbase::RpcClient> rpc_client, std::shared_ptr<Scan> scan, int64_t scanner_id,
-    std::shared_ptr<ScanResultCache> results_cache, std::shared_ptr<RawScanResultConsumer> consumer,
-    std::shared_ptr<RegionLocation> region_location, nanoseconds scanner_lease_timeout_nanos,
-    nanoseconds pause, uint32_t max_retries, nanoseconds scan_timeout_nanos,
-    nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count)
-    : conn_(conn),
-      retry_timer_(retry_timer),
-      rpc_client_(rpc_client),
-      scan_(scan),
-      scanner_id_(scanner_id),
-      results_cache_(results_cache),
-      consumer_(consumer),
-      region_location_(region_location),
-      scanner_lease_timeout_nanos_(scanner_lease_timeout_nanos),
-      pause_(pause),
-      max_retries_(max_retries),
-      scan_timeout_nanos_(scan_timeout_nanos),
-      rpc_timeout_nanos_(rpc_timeout_nanos),
-      start_log_errors_count_(start_log_errors_count),
-      promise_(std::make_shared<folly::Promise<bool>>()),
-      tries_(1) {
-  controller_ = conn_->CreateRpcController();
-  start_ns_ = TimeUtil::GetNowNanos();
-  max_attempts_ = ConnectionUtils::Retries2Attempts(max_retries);
-  exceptions_ = std::make_shared<std::vector<ThrowableWithExtraContext>>();
-}
-
-folly::Future<bool> AsyncScanRpcRetryingCaller::Start(
-    std::shared_ptr<HBaseRpcController> controller,
-    std::shared_ptr<pb::ScanResponse> open_scan_resp,
-    const std::shared_ptr<CellScanner> cell_scanner) {
-  OnComplete(controller, open_scan_resp, cell_scanner);
-  return promise_->getFuture();
-}
-
-int64_t AsyncScanRpcRetryingCaller::RemainingTimeNs() {
-  return scan_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
-}
-
-void AsyncScanRpcRetryingCaller::OnComplete(std::shared_ptr<HBaseRpcController> controller,
-                                            std::shared_ptr<pb::ScanResponse> resp,
-                                            const std::shared_ptr<CellScanner> cell_scanner) {
-  VLOG(5) << "Scan: OnComplete, scanner_id:" << scanner_id_;
-
-  if (controller->Failed()) {
-    OnError(controller->exception());
-    return;
-  }
-
-  bool is_heartbeat = resp->has_heartbeat_message() && resp->heartbeat_message();
-
-  int64_t num_complete_rows_before = results_cache_->num_complete_rows();
-  try {
-    auto raw_results = ResponseConverter::FromScanResponse(resp, cell_scanner);
-
-    auto results = results_cache_->AddAndGet(raw_results, is_heartbeat);
-
-    auto scan_controller = std::make_shared<ScanControllerImpl>(shared_from_this());
-
-    if (results.size() > 0) {
-      UpdateNextStartRowWhenError(*results[results.size() - 1]);
-      VLOG(5) << "Calling consumer->OnNext()";
-      consumer_->OnNext(results, scan_controller);
-    } else if (is_heartbeat) {
-      consumer_->OnHeartbeat(scan_controller);
-    }
-
-    ScanControllerState state = scan_controller->Destroy();
-    if (state == ScanControllerState::kTerminated) {
-      if (resp->has_more_results_in_region() && !resp->more_results_in_region()) {
-        // we have more results in region but user request to stop the scan, so we need to close the
-        // scanner explicitly.
-        CloseScanner();
-      }
-      CompleteNoMoreResults();
-      return;
-    }
-
-    int64_t num_complete_rows = results_cache_->num_complete_rows() - num_complete_rows_before;
-    if (state == ScanControllerState::kSuspended) {
-      if (scan_controller->resumer()->Prepare(resp, num_complete_rows)) {
-        return;
-      }
-    }
-  } catch (const std::runtime_error& e) {
-    // We can not retry here. The server has responded normally and the call sequence has been
-    // increased so a new scan with the same call sequence will cause an
-    // OutOfOrderScannerNextException. Let the upper layer open a new scanner.
-    LOG(WARNING) << "Received exception in reading the scan response:" << e.what();
-    CompleteWhenError(true);
-    return;
-  }
-
-  CompleteOrNext(resp);
-}
-
-void AsyncScanRpcRetryingCaller::CompleteOrNext(std::shared_ptr<pb::ScanResponse> resp) {
-  VLOG(5) << "Scan: CompleteOrNext, scanner_id" << scanner_id_
-          << ", response:" << resp->ShortDebugString();
-
-  if (resp->has_more_results() && !resp->more_results()) {
-    // RS tells us there is no more data for the whole scan
-    CompleteNoMoreResults();
-    return;
-  }
-  // TODO: Implement Scan::limit(), and check the limit here
-
-  if (resp->has_more_results_in_region() && !resp->more_results_in_region()) {
-    // TODO: check whether Scan is reversed here
-    CompleteWhenNoMoreResultsInRegion();
-    return;
-  }
-  Next();
-}
-
-void AsyncScanRpcRetryingCaller::CompleteExceptionally(bool close_scanner) {
-  VLOG(5) << "Scan: CompleteExceptionally";
-  results_cache_->Clear();
-  if (close_scanner) {
-    CloseScanner();
-  }
-  this->promise_->setException(RetriesExhaustedException(tries_ - 1, exceptions_));
-}
-
-void AsyncScanRpcRetryingCaller::CompleteNoMoreResults() {
-  // In master code, scanners auto-close if we have exhausted the region. It may not be the case
-  // in branch-1 code. If this is backported, make sure that the scanner is closed.
-  VLOG(5) << "Scan: CompleteNoMoreResults, scanner_id:" << scanner_id_;
-  promise_->setValue(false);
-}
-
-void AsyncScanRpcRetryingCaller::CompleteWhenNoMoreResultsInRegion() {
-  VLOG(5) << "Scan: CompleteWhenNoMoreResultsInRegion, scanner_id:" << scanner_id_;
-  // In master code, scanners auto-close if we have exhausted the region. It may not be the case
-  // in branch-1 code. If this is backported, make sure that the scanner is closed.
-  if (NoMoreResultsForScan(*scan_, region_location_->region_info())) {
-    CompleteNoMoreResults();
-  } else {
-    CompleteWithNextStartRow(region_location_->region_info().end_key(), true);
-  }
-}
-
-void AsyncScanRpcRetryingCaller::CompleteWithNextStartRow(std::string row, bool inclusive) {
-  VLOG(5) << "Scan: CompleteWithNextStartRow: region scan is complete, move to next region";
-  scan_->SetStartRow(row);
-  // TODO: set inclusive if it is reverse scans
-  promise_->setValue(true);
-}
-
-void AsyncScanRpcRetryingCaller::UpdateNextStartRowWhenError(const Result& result) {
-  next_start_row_when_error_ = optional<std::string>(result.Row());
-  include_next_start_row_when_error_ = result.Partial();
-}
-
-void AsyncScanRpcRetryingCaller::CompleteWhenError(bool close_scanner) {
-  VLOG(5) << "Scan: CompleteWhenError, scanner_id:" << scanner_id_;
-  results_cache_->Clear();
-  if (close_scanner) {
-    CloseScanner();
-  }
-  if (next_start_row_when_error_) {
-    // TODO: HBASE-17583 adds include start / stop row to the Scan. Once we rebase and implement
-    // those options in Scan , we can start using that here.
-    scan_->SetStartRow(include_next_start_row_when_error_
-                           ? *next_start_row_when_error_
-                           : BytesUtil::CreateClosestRowAfter(*next_start_row_when_error_));
-  }
-  promise_->setValue(true);
-}
-
-void AsyncScanRpcRetryingCaller::OnError(const folly::exception_wrapper& error) {
-  VLOG(5) << "Scan: OnError, scanner_id:" << scanner_id_;
-  if (tries_ > start_log_errors_count_ || VLOG_IS_ON(5)) {
-    LOG(WARNING) << "Call to " << region_location_->server_name().ShortDebugString()
-                 << " for scanner id = " << scanner_id_ << " for "
-                 << region_location_->region_info().ShortDebugString()
-                 << " failed, , tries = " << tries_ << ", maxAttempts = " << max_attempts_
-                 << ", timeout = " << TimeUtil::ToMillis(scan_timeout_nanos_).count()
-                 << " ms, time elapsed = " << TimeUtil::ElapsedMillis(start_ns_) << " ms"
-                 << error.what().toStdString();
-  }
-
-  bool scanner_closed = ExceptionUtil::IsScannerClosed(error);
-  ThrowableWithExtraContext twec(error, TimeUtil::GetNowNanos());
-  exceptions_->push_back(twec);
-  if (tries_ >= max_retries_) {
-    CompleteExceptionally(!scanner_closed);
-    return;
-  }
-
-  int64_t delay_ns;
-  if (scan_timeout_nanos_.count() > 0) {
-    int64_t max_delay_ns = RemainingTimeNs() - ConnectionUtils::kSleepDeltaNs;
-    if (max_delay_ns <= 0) {
-      CompleteExceptionally(!scanner_closed);
-      return;
-    }
-    delay_ns = std::min(max_delay_ns, ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1));
-  } else {
-    delay_ns = ConnectionUtils::GetPauseTime(pause_.count(), tries_ - 1);
-  }
-
-  if (scanner_closed) {
-    CompleteWhenError(false);
-    return;
-  }
-
-  if (ExceptionUtil::IsScannerOutOfOrder(error)) {
-    CompleteWhenError(true);
-    return;
-  }
-  if (!ExceptionUtil::ShouldRetry(error)) {
-    CompleteExceptionally(true);
-    return;
-  }
-  tries_++;
-
-  auto self(shared_from_this());
-  conn_->retry_executor()->add([&]() {
-    retry_timer_->scheduleTimeoutFn(
-        [self]() { self->conn_->cpu_executor()->add([&]() { self->Call(); }); },
-        std::chrono::milliseconds(TimeUtil::ToMillis(delay_ns)));
-  });
-}
-
-bool AsyncScanRpcRetryingCaller::NoMoreResultsForScan(const Scan& scan,
-                                                      const pb::RegionInfo& info) {
-  if (BytesUtil::IsEmptyStopRow(info.end_key())) {
-    return true;
-  }
-  if (BytesUtil::IsEmptyStopRow(scan.StopRow())) {
-    return false;
-  }
-  int32_t c = BytesUtil::CompareTo(info.end_key(), scan.StopRow());
-  // 1. if our stop row is less than the endKey of the region
-  // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
-  // for scan.
-  return c > 0 ||
-         (c == 0 /* && !scan.IncludeStopRow()*/);  // TODO: Scans always exclude StopRow for now.
-}
-
-void AsyncScanRpcRetryingCaller::Next() {
-  VLOG(5) << "Scan: Next";
-  next_call_seq_++;
-  tries_ = 1;
-  exceptions_->clear();
-  start_ns_ = TimeUtil::GetNowNanos();
-  Call();
-}
-
-void AsyncScanRpcRetryingCaller::Call() {
-  VLOG(5) << "Scan: Call";
-  auto self(shared_from_this());
-  // As we have a call sequence for scan, it is useless to have a different rpc timeout which is
-  // less than the scan timeout. If the server does not respond in time(usually this will not
-  // happen as we have heartbeat now), we will get an OutOfOrderScannerNextException when
-  // resending the next request and the only way to fix this is to close the scanner and open a
-  // new one.
-  int64_t call_timeout_nanos;
-  if (scan_timeout_nanos_.count() > 0) {
-    int64_t remaining_nanos = scan_timeout_nanos_.count() - (TimeUtil::GetNowNanos() - start_ns_);
-    if (remaining_nanos <= 0) {
-      CompleteExceptionally(true);
-      return;
-    }
-    call_timeout_nanos = remaining_nanos;
-  } else {
-    call_timeout_nanos = 0L;
-  }
-
-  ResetController(controller_, call_timeout_nanos);
-
-  auto req =
-      RequestConverter::ToScanRequest(scanner_id_, scan_->Caching(), false, next_call_seq_, false);
-
-  // do the RPC call
-  rpc_client_
-      ->AsyncCall(region_location_->server_name().host_name(),
-                  region_location_->server_name().port(), std::move(req),
-                  security::User::defaultUser(), "ClientService")
-      .via(conn_->cpu_executor().get())
-      .then([self, this](const std::unique_ptr<Response>& resp) {
-        auto scan_resp = std::static_pointer_cast<pb::ScanResponse>(resp->resp_msg());
-        return OnComplete(controller_, scan_resp, resp->cell_scanner());
-      })
-      .onError([self, this](const folly::exception_wrapper& e) { OnError(e); });
-}
-
-void AsyncScanRpcRetryingCaller::CloseScanner() {
-  auto self(shared_from_this());
-  ResetController(controller_, rpc_timeout_nanos_.count());
-
-  VLOG(5) << "Closing scanner with scanner_id:" << folly::to<std::string>(scanner_id_);
-
-  // Do a close scanner RPC. Fire and forget.
-  auto req = RequestConverter::ToScanRequest(scanner_id_, 0, true);
-  rpc_client_
-      ->AsyncCall(region_location_->server_name().host_name(),
-                  region_location_->server_name().port(), std::move(req),
-                  security::User::defaultUser(), "ClientService")
-      .onError([self, this](const folly::exception_wrapper& e) -> std::unique_ptr<Response> {
-        LOG(WARNING) << "Call to " + region_location_->server_name().ShortDebugString() +
-                            " for closing scanner_id = " + folly::to<std::string>(scanner_id_) +
-                            " for " + region_location_->region_info().ShortDebugString() +
-                            " failed, ignore, probably already closed. Exception:" +
-                            e.what().toStdString();
-        return nullptr;
-      });
-}
-
-void AsyncScanRpcRetryingCaller::ResetController(std::shared_ptr<HBaseRpcController> controller,
-                                                 const int64_t& timeout_nanos) {
-  controller->Reset();
-  if (timeout_nanos >= 0) {
-    controller->set_call_timeout(
-        milliseconds(std::min(static_cast<int64_t>(INT_MAX), TimeUtil::ToMillis(timeout_nanos))));
-  }
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-scan-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-scan-rpc-retrying-caller.h b/hbase-native-client/core/async-scan-rpc-retrying-caller.h
deleted file mode 100644
index 9555e80..0000000
--- a/hbase-native-client/core/async-scan-rpc-retrying-caller.h
+++ /dev/null
@@ -1,233 +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.
- *
- */
-#pragma once
-
-#include <folly/Conv.h>
-#include <folly/ExceptionWrapper.h>
-#include <folly/Format.h>
-#include <folly/Logging.h>
-#include <folly/futures/Future.h>
-#include <folly/io/async/EventBase.h>
-#include <folly/io/async/HHWheelTimer.h>
-
-#include <algorithm>
-#include <chrono>
-#include <functional>
-#include <memory>
-#include <string>
-#include <type_traits>
-#include <utility>
-#include <vector>
-
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/hbase-rpc-controller.h"
-#include "core/raw-scan-result-consumer.h"
-#include "core/region-location.h"
-#include "core/request-converter.h"
-#include "core/response-converter.h"
-#include "core/result.h"
-#include "core/scan-result-cache.h"
-#include "core/scan.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-#include "utils/bytes-util.h"
-#include "utils/connection-util.h"
-#include "utils/optional.h"
-#include "utils/sys-util.h"
-#include "utils/time-util.h"
-
-using std::chrono::nanoseconds;
-using std::chrono::milliseconds;
-
-namespace hbase {
-
-class AsyncScanRpcRetryingCaller;
-
-// The resume method is allowed to be called in another thread so here we also use the
-// ResumerState to prevent race. The initial state is INITIALIZED, and in most cases, when back
-// from onNext or onHeartbeat, we will call the prepare method to change the state to SUSPENDED,
-// and when user calls resume method, we will change the state to RESUMED. But the resume method
-// could be called in other thread, and in fact, user could just do this:
-// controller.suspend().resume()
-// This is strange but valid. This means the scan could be resumed before we call the prepare
-// method to do the actual suspend work. So in the resume method, we will check if the state is
-// INTIALIZED, if it is, then we will just set the state to RESUMED and return. And in prepare
-// method, if the state is RESUMED already, we will just return an let the scan go on.
-// Notice that, the public methods of this class is supposed to be called by upper layer only, and
-// package private methods can only be called within the implementation of
-// AsyncScanSingleRegionRpcRetryingCaller.
-// TODO: Unlike the Java counter part, we do not do scan lease renewals in a background thread.
-// Since there is also no async scan API exposed to the users, only ScanResultConsumer is the
-// AsyncTableResultScanner which will only pause the scanner if the result cache is maxed. The
-// application is expected to consume the scan results before the scanner lease timeout.
-class ScanResumerImpl : public ScanResumer {
- public:
-  explicit ScanResumerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller);
-
-  virtual ~ScanResumerImpl() = default;
-
-  /**
-   * Resume the scan. You are free to call it multiple time but only the first call will take
-   * effect.
-   */
-  void Resume() override;
-
-  // return false if the scan has already been resumed. See the comment above for ScanResumerImpl
-  // for more details.
-  bool Prepare(std::shared_ptr<pb::ScanResponse> resp, int num_complete_rows);
-
- private:
-  // INITIALIZED -> SUSPENDED -> RESUMED
-  // INITIALIZED -> RESUMED
-  ScanResumerState state_ = ScanResumerState::kInitialized;
-  std::mutex mutex_;
-  std::shared_ptr<pb::ScanResponse> resp_ = nullptr;
-  int64_t num_complete_rows_ = 0;
-  std::shared_ptr<AsyncScanRpcRetryingCaller> caller_;
-};
-
-class ScanControllerImpl : public ScanController {
- public:
-  virtual ~ScanControllerImpl() = default;
-
-  explicit ScanControllerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller);
-
-  /**
-   * Suspend the scan.
-   * <p>
-   * This means we will stop fetching data in background, i.e., will not call onNext any more
-   * before you resume the scan.
-   * @return A resumer used to resume the scan later.
-   */
-  std::shared_ptr<ScanResumer> Suspend();
-
-  /**
-   * Terminate the scan.
-   * <p>
-   * This is useful when you have got enough results and want to stop the scan in onNext method,
-   * or you want to stop the scan in onHeartbeat method because it has spent too many time.
-   */
-  void Terminate();
-
-  // return the current state, and set the state to DESTROYED.
-  ScanControllerState Destroy();
-
-  std::shared_ptr<ScanResumerImpl> resumer() { return resumer_; }
-
- private:
-  void PreCheck();
-
-  std::string DebugString(ScanControllerState state);
-
-  std::string DebugString(ScanResumerState state);
-
- private:
-  // Make sure the methods are only called in this thread.
-  std::thread::id caller_thread_id_ = std::this_thread::get_id();
-  // INITIALIZED -> SUSPENDED -> DESTROYED
-  // INITIALIZED -> TERMINATED -> DESTROYED
-  // INITIALIZED -> DESTROYED
-  // If the state is incorrect we will throw IllegalStateException.
-  ScanControllerState state_ = ScanControllerState::kInitialized;
-  std::shared_ptr<ScanResumerImpl> resumer_ = nullptr;
-  std::shared_ptr<AsyncScanRpcRetryingCaller> caller_;
-};
-
-class AsyncScanRpcRetryingCaller : public std::enable_shared_from_this<AsyncScanRpcRetryingCaller> {
- public:
-  AsyncScanRpcRetryingCaller(std::shared_ptr<AsyncConnection> conn,
-                             std::shared_ptr<folly::HHWheelTimer> retry_timer,
-                             std::shared_ptr<hbase::RpcClient> rpc_client,
-                             std::shared_ptr<Scan> scan, int64_t scanner_id,
-                             std::shared_ptr<ScanResultCache> results_cache,
-                             std::shared_ptr<RawScanResultConsumer> consumer,
-                             std::shared_ptr<RegionLocation> region_location,
-                             nanoseconds scanner_lease_timeout_nanos, nanoseconds pause,
-                             uint32_t max_retries, nanoseconds scan_timeout_nanos,
-                             nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
-
-  folly::Future<bool> Start(std::shared_ptr<HBaseRpcController> controller,
-                            std::shared_ptr<pb::ScanResponse> open_scan_resp,
-                            const std::shared_ptr<CellScanner> cell_scanner);
-
- private:
-  int64_t RemainingTimeNs();
-  void OnComplete(std::shared_ptr<HBaseRpcController> controller,
-                  std::shared_ptr<pb::ScanResponse> resp,
-                  const std::shared_ptr<CellScanner> cell_scanner);
-
-  void CompleteOrNext(std::shared_ptr<pb::ScanResponse> resp);
-
-  void CompleteExceptionally(bool close_scanner);
-
-  void CompleteNoMoreResults();
-
-  void CompleteWhenNoMoreResultsInRegion();
-
-  void CompleteWithNextStartRow(std::string row, bool inclusive);
-
-  void UpdateNextStartRowWhenError(const Result& result);
-
-  void CompleteWhenError(bool close_scanner);
-
-  void OnError(const folly::exception_wrapper& e);
-
-  bool NoMoreResultsForScan(const Scan& scan, const pb::RegionInfo& info);
-
-  void Next();
-
-  void Call();
-
-  void CloseScanner();
-
-  void ResetController(std::shared_ptr<HBaseRpcController> controller,
-                       const int64_t& timeout_nanos);
-
- private:
-  std::shared_ptr<AsyncConnection> conn_;
-  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
-  std::shared_ptr<hbase::RpcClient> rpc_client_;
-  std::shared_ptr<Scan> scan_;
-  int64_t scanner_id_;
-  std::shared_ptr<ScanResultCache> results_cache_;
-  std::shared_ptr<RawScanResultConsumer> consumer_;
-  std::shared_ptr<RegionLocation> region_location_;
-  nanoseconds scanner_lease_timeout_nanos_;
-  nanoseconds pause_;
-  uint32_t max_retries_;
-  nanoseconds scan_timeout_nanos_;
-  nanoseconds rpc_timeout_nanos_;
-  uint32_t start_log_errors_count_;
-  std::shared_ptr<folly::Promise<bool>> promise_;
-  std::shared_ptr<HBaseRpcController> controller_;
-  optional<std::string> next_start_row_when_error_ = optional<std::string>();
-  bool include_next_start_row_when_error_ = true;
-  uint64_t start_ns_;
-  uint32_t tries_;
-  std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions_;
-  uint32_t max_attempts_;
-  int64_t next_call_seq_ = -1L;
-
-  friend class ScanResumerImpl;
-  friend class ScanControllerImpl;
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-table-result-scanner.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-table-result-scanner.cc b/hbase-native-client/core/async-table-result-scanner.cc
deleted file mode 100644
index b1935ae..0000000
--- a/hbase-native-client/core/async-table-result-scanner.cc
+++ /dev/null
@@ -1,161 +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/async-table-result-scanner.h"
-
-#include <vector>
-
-namespace hbase {
-AsyncTableResultScanner::AsyncTableResultScanner(int64_t max_cache_size)
-    : max_cache_size_(max_cache_size) {
-  closed_ = false;
-  cache_size_ = 0;
-}
-
-AsyncTableResultScanner::~AsyncTableResultScanner() { Close(); }
-
-void AsyncTableResultScanner::Close() {
-  std::unique_lock<std::mutex> mlock(mutex_);
-  closed_ = true;
-  while (!queue_.empty()) {
-    queue_.pop();
-  }
-  cache_size_ = 0;
-  if (resumer_ != nullptr) {
-    resumer_->Resume();
-  }
-  cond_.notify_all();
-}
-
-std::shared_ptr<Result> AsyncTableResultScanner::Next() {
-  VLOG(5) << "AsyncTableResultScanner: Next()";
-
-  std::shared_ptr<Result> result = nullptr;
-  std::shared_ptr<ScanResumer> local_resumer = nullptr;
-  {
-    std::unique_lock<std::mutex> mlock(mutex_);
-    while (queue_.empty()) {
-      if (closed_) {
-        return nullptr;
-      }
-      if (error_) {
-        throw error_;
-      }
-      cond_.wait(mlock);
-    }
-    result = queue_.front();
-    queue_.pop();
-
-    cache_size_ -= EstimatedSizeWithSharedPtr(result);
-    if (resumer_ != nullptr && cache_size_ <= max_cache_size_ / 2) {
-      VLOG(1) << std::this_thread::get_id() << " resume scan prefetching";
-      local_resumer = resumer_;
-      resumer_ = nullptr;
-    }
-  }
-
-  // Need to call ScanResumer::Resume() outside of the scope of the mutex. The reason is that
-  // folly/wangle event loop might end up running the attached logic(.then()) at the Scan RPC
-  // in the same event thread before returning from the previous call. This seems like the
-  // wrong thing to do(â„¢), but we cannot fix that now. Since the call back can end up calling
-  // this::OnNext(), we should unlock the mutex.
-  if (local_resumer != nullptr) {
-    local_resumer->Resume();
-  }
-  return result;
-}
-
-void AsyncTableResultScanner::AddToCache(const std::vector<std::shared_ptr<Result>> &results) {
-  VLOG(5) << "AsyncTableResultScanner: AddToCache()";
-  for (const auto r : results) {
-    queue_.push(r);
-    cache_size_ += EstimatedSizeWithSharedPtr(r);
-  }
-}
-
-template <typename T>
-inline size_t AsyncTableResultScanner::EstimatedSizeWithSharedPtr(std::shared_ptr<T> t) {
-  return t->EstimatedSize() + sizeof(std::shared_ptr<T>);
-}
-
-void AsyncTableResultScanner::OnNext(const std::vector<std::shared_ptr<Result>> &results,
-                                     std::shared_ptr<ScanController> controller) {
-  VLOG(5) << "AsyncTableResultScanner: OnNext()";
-  {
-    std::unique_lock<std::mutex> mlock(mutex_);
-    if (closed_) {
-      controller->Terminate();
-      return;
-    }
-    AddToCache(results);
-
-    if (cache_size_ >= max_cache_size_) {
-      StopPrefetch(controller);
-    }
-  }
-  cond_.notify_all();
-}
-
-void AsyncTableResultScanner::StopPrefetch(std::shared_ptr<ScanController> controller) {
-  VLOG(1) << std::this_thread::get_id()
-          << ": stop prefetching when scanning as the cache size " +
-                 folly::to<std::string>(cache_size_) + " is greater than the max_cache_size " +
-                 folly::to<std::string>(max_cache_size_);
-
-  resumer_ = controller->Suspend();
-  num_prefetch_stopped_++;
-}
-
-/**
- * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
- * onNext.
- * <p>
- * This method give you a chance to terminate a slow scan operation.
- * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
- *          instance is only valid within the scope of onHeartbeat method. You can only call its
- *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
- */
-void AsyncTableResultScanner::OnHeartbeat(std::shared_ptr<ScanController> controller) {
-  std::unique_lock<std::mutex> mlock(mutex_);
-  if (closed_) {
-    controller->Terminate();
-  }
-}
-
-/**
- * Indicate that we hit an unrecoverable error and the scan operation is terminated.
- * <p>
- * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
- */
-void AsyncTableResultScanner::OnError(const folly::exception_wrapper &error) {
-  LOG(WARNING) << "Scanner received error" << error.what();
-  std::unique_lock<std::mutex> mlock(mutex_);
-  error_ = error;
-  cond_.notify_all();
-}
-
-/**
- * Indicate that the scan operation is completed normally.
- */
-void AsyncTableResultScanner::OnComplete() {
-  std::unique_lock<std::mutex> mlock(mutex_);
-  closed_ = true;
-  cond_.notify_all();
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/async-table-result-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/async-table-result-scanner.h b/hbase-native-client/core/async-table-result-scanner.h
deleted file mode 100644
index dcdf871..0000000
--- a/hbase-native-client/core/async-table-result-scanner.h
+++ /dev/null
@@ -1,98 +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.
- *
- */
-#pragma once
-
-#include <folly/Conv.h>
-#include <folly/ExceptionWrapper.h>
-#include <folly/Logging.h>
-#include <chrono>
-#include <condition_variable>
-#include <memory>
-#include <mutex>
-#include <queue>
-#include <string>
-#include <vector>
-
-#include "core/raw-scan-result-consumer.h"
-#include "core/result-scanner.h"
-#include "core/result.h"
-#include "if/Client.pb.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-class AsyncTableResultScanner : public ResultScanner, public RawScanResultConsumer {
- public:
-  explicit AsyncTableResultScanner(int64_t max_cache_size);
-
-  virtual ~AsyncTableResultScanner();
-
-  void Close() override;
-
-  std::shared_ptr<Result> Next() override;
-
-  void OnNext(const std::vector<std::shared_ptr<Result>> &results,
-              std::shared_ptr<ScanController> controller) override;
-
-  /**
-   * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
-   * onNext.
-   * <p>
-   * This method give you a chance to terminate a slow scan operation.
-   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
-   *          instance is only valid within the scope of onHeartbeat method. You can only call its
-   *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
-   */
-  void OnHeartbeat(std::shared_ptr<ScanController> controller) override;
-
-  /**
-   * Indicate that we hit an unrecoverable error and the scan operation is terminated.
-   * <p>
-   * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
-   */
-  void OnError(const folly::exception_wrapper &error) override;
-
-  /**
-   * Indicate that the scan operation is completed normally.
-   */
-  void OnComplete() override;
-
-  // For testing
-  uint32_t num_prefetch_stopped() { return num_prefetch_stopped_; }
-
- private:
-  void AddToCache(const std::vector<std::shared_ptr<Result>> &results);
-
-  template <typename T>
-  inline size_t EstimatedSizeWithSharedPtr(std::shared_ptr<T> t);
-
-  void StopPrefetch(std::shared_ptr<ScanController> controller);
-
- private:
-  std::queue<std::shared_ptr<Result>> queue_;
-  std::mutex mutex_;
-  std::condition_variable cond_;
-  folly::exception_wrapper error_;
-  int64_t cache_size_;
-  int64_t max_cache_size_;
-  bool closed_;
-  std::shared_ptr<ScanResumer> resumer_ = nullptr;
-  uint32_t num_prefetch_stopped_ = 0;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/cell-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell-test.cc b/hbase-native-client/core/cell-test.cc
deleted file mode 100644
index 4611473..0000000
--- a/hbase-native-client/core/cell-test.cc
+++ /dev/null
@@ -1,195 +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/cell.h"
-
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-#include <memory>
-
-using hbase::Cell;
-using hbase::CellType;
-
-TEST(CellTest, Constructor) {
-  std::string row = "row-value";
-  std::string family = "family-value";
-  std::string column = "column-value";
-  std::string value = "value-value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-
-  Cell cell{row, family, column, timestamp, value, cell_type};
-
-  EXPECT_EQ(row, cell.Row());
-  EXPECT_EQ(family, cell.Family());
-  EXPECT_EQ(column, cell.Qualifier());
-  EXPECT_EQ(value, cell.Value());
-  EXPECT_EQ(timestamp, cell.Timestamp());
-  EXPECT_EQ(cell_type, cell.Type());
-}
-
-TEST(CellTest, CopyConstructor) {
-  std::string row = "row-value";
-  std::string family = "family-value";
-  std::string column = "column-value";
-  std::string value = "value-value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-  Cell cell2{*cell};
-  cell = nullptr;
-
-  EXPECT_EQ(row, cell2.Row());
-  EXPECT_EQ(family, cell2.Family());
-  EXPECT_EQ(column, cell2.Qualifier());
-  EXPECT_EQ(value, cell2.Value());
-  EXPECT_EQ(timestamp, cell2.Timestamp());
-  EXPECT_EQ(cell_type, cell2.Type());
-}
-
-TEST(CellTest, CopyAssignment) {
-  std::string row = "row-value";
-  std::string family = "family-value";
-  std::string column = "column-value";
-  std::string value = "value-value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-  Cell cell2 = *cell;
-  cell = nullptr;
-
-  EXPECT_EQ(row, cell2.Row());
-  EXPECT_EQ(family, cell2.Family());
-  EXPECT_EQ(column, cell2.Qualifier());
-  EXPECT_EQ(value, cell2.Value());
-  EXPECT_EQ(timestamp, cell2.Timestamp());
-  EXPECT_EQ(cell_type, cell2.Type());
-}
-
-TEST(CellTest, CellRowTest) {
-  std::string row = "only-row";
-  std::string family = "D";
-  std::string column = "";
-  std::string value = "";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-  Cell cell{row, family, column, timestamp, value, cell_type};
-
-  EXPECT_EQ(row, cell.Row());
-  EXPECT_EQ(family, cell.Family());
-  EXPECT_EQ(column, cell.Qualifier());
-  EXPECT_EQ(value, cell.Value());
-  EXPECT_EQ(timestamp, cell.Timestamp());
-  EXPECT_EQ(cell_type, cell.Type());
-}
-
-TEST(CellTest, CellRowFamilyTest) {
-  std::string row = "only-row";
-  std::string family = "only-family";
-  std::string column = "";
-  std::string value = "";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-  Cell cell{row, family, column, timestamp, value, cell_type};
-
-  EXPECT_EQ(row, cell.Row());
-  EXPECT_EQ(family, cell.Family());
-  EXPECT_EQ(column, cell.Qualifier());
-  EXPECT_EQ(value, cell.Value());
-  EXPECT_EQ(timestamp, cell.Timestamp());
-  EXPECT_EQ(cell_type, cell.Type());
-}
-
-TEST(CellTest, CellRowFamilyValueTest) {
-  std::string row = "only-row";
-  std::string family = "only-family";
-  std::string column = "";
-  std::string value = "only-value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-
-  Cell cell{row, family, column, timestamp, value, cell_type};
-
-  EXPECT_EQ(row, cell.Row());
-  EXPECT_EQ(family, cell.Family());
-  EXPECT_EQ(column, cell.Qualifier());
-  EXPECT_EQ(value, cell.Value());
-  EXPECT_EQ(timestamp, cell.Timestamp());
-  EXPECT_EQ(cell_type, cell.Type());
-}
-
-TEST(CellTest, CellRowFamilyColumnValueTest) {
-  std::string row = "only-row";
-  std::string family = "only-family";
-  std::string column = "only-column";
-  std::string value = "only-value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  CellType cell_type = CellType::PUT;
-  Cell cell{row, family, column, timestamp, value, cell_type};
-
-  EXPECT_EQ(row, cell.Row());
-  EXPECT_EQ(family, cell.Family());
-  EXPECT_EQ(column, cell.Qualifier());
-  EXPECT_EQ(value, cell.Value());
-  EXPECT_EQ(timestamp, cell.Timestamp());
-  EXPECT_EQ(cell_type, cell.Type());
-}
-
-TEST(CellTest, CellDebugString) {
-  CellType cell_type = CellType::PUT;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-
-  Cell cell{row, family, column, timestamp, value, cell_type};
-  LOG(INFO) << cell.DebugString();
-  EXPECT_EQ("row/family:column/LATEST_TIMESTAMP/PUT/vlen=5/seqid=0", cell.DebugString());
-
-  Cell cell2{row, "", column, 42, value, CellType::DELETE};
-  LOG(INFO) << cell2.DebugString();
-  EXPECT_EQ("row/column/42/DELETE/vlen=5/seqid=0", cell2.DebugString());
-}
-
-TEST(CellTest, CellEstimatedSize) {
-  CellType cell_type = CellType::PUT;
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-
-  Cell empty{"a", "a", "", timestamp, "", cell_type};
-  Cell cell1{"aa", "a", "", timestamp, "", cell_type};
-  Cell cell2{"a", "aa", "", timestamp, "", cell_type};
-  Cell cell3{"a", "a", "a", timestamp, "", cell_type};
-  Cell cell4{"a", "a", "", timestamp, "a", cell_type};
-  Cell cell5{"a", "a", "", timestamp, "a", CellType::DELETE};
-  Cell cell6{"aaaaaa", "a", "", timestamp, "a", cell_type};
-
-  LOG(INFO) << empty.EstimatedSize();
-  LOG(INFO) << cell1.EstimatedSize();
-
-  EXPECT_TRUE(empty.EstimatedSize() > sizeof(Cell));
-  EXPECT_TRUE(cell1.EstimatedSize() > empty.EstimatedSize());
-  EXPECT_EQ(cell1.EstimatedSize(), cell2.EstimatedSize());
-  EXPECT_EQ(cell2.EstimatedSize(), cell3.EstimatedSize());
-  EXPECT_EQ(cell3.EstimatedSize(), cell4.EstimatedSize());
-  EXPECT_EQ(cell4.EstimatedSize(), cell5.EstimatedSize());
-  EXPECT_TRUE(cell6.EstimatedSize() > cell1.EstimatedSize());
-}


[14/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Snapshot.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Snapshot.proto b/hbase-native-client/if/Snapshot.proto
deleted file mode 100644
index ae1a1e6..0000000
--- a/hbase-native-client/if/Snapshot.proto
+++ /dev/null
@@ -1,66 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "SnapshotProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "FS.proto";
-import "HBase.proto";
-
-message SnapshotFileInfo {
-  enum Type {
-    HFILE = 1;
-    WAL = 2;
-  }
-
-  required Type type = 1;
-
-  optional string hfile = 3;
-
-  optional string wal_server = 4;
-  optional string wal_name = 5;
-}
-
-message SnapshotRegionManifest {
-  optional int32 version = 1;
-
-  required RegionInfo region_info = 2;
-  repeated FamilyFiles family_files = 3;
-
-  message StoreFile {
-    required string name = 1;
-    optional Reference reference = 2;
-
-    // TODO: Add checksums or other fields to verify the file
-    optional uint64 file_size = 3;
-  }
-
-  message FamilyFiles {
-    required bytes family_name = 1;
-    repeated StoreFile store_files = 2;
-  }
-}
-
-message SnapshotDataManifest {
-  required TableSchema table_schema = 1;
-  repeated SnapshotRegionManifest region_manifests = 2;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/Tracing.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/Tracing.proto b/hbase-native-client/if/Tracing.proto
deleted file mode 100644
index 5a64cfc..0000000
--- a/hbase-native-client/if/Tracing.proto
+++ /dev/null
@@ -1,33 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "TracingProtos";
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-//Used to pass through the information necessary to continue
-//a trace after an RPC is made. All we need is the traceid 
-//(so we know the overarching trace this message is a part of), and
-//the id of the current span when this message was sent, so we know 
-//what span caused the new span we will create when this message is received.
-message RPCTInfo {
-  optional int64 trace_id = 1;
-  optional int64 parent_id = 2;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/VisibilityLabels.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/VisibilityLabels.proto b/hbase-native-client/if/VisibilityLabels.proto
deleted file mode 100644
index d2dc44d..0000000
--- a/hbase-native-client/if/VisibilityLabels.proto
+++ /dev/null
@@ -1,83 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "VisibilityLabelsProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "Client.proto";
-
-message VisibilityLabelsRequest {
-  repeated VisibilityLabel visLabel = 1;
-}
-
-message VisibilityLabel {
-  required bytes label = 1;
-  optional uint32 ordinal = 2;
-}
-
-message VisibilityLabelsResponse {
-  repeated RegionActionResult result = 1; 
-}
-
-message SetAuthsRequest {
-  required bytes user = 1;
-  repeated bytes auth = 2;
-}
-
-message UserAuthorizations {
-  required bytes user = 1;
-  repeated uint32 auth = 2;
-}
-
-message MultiUserAuthorizations {
-  repeated UserAuthorizations userAuths = 1;
-}
-
-message GetAuthsRequest {
-  required bytes user = 1;
-}
-
-message GetAuthsResponse {
-  required bytes user = 1;
-  repeated bytes auth = 2;
-}
-
-message ListLabelsRequest {
-  optional string regex = 1;
-}
-
-message ListLabelsResponse {
-  repeated bytes label = 1;
-}
-
-service VisibilityLabelsService {
-  rpc addLabels(VisibilityLabelsRequest)
-    returns (VisibilityLabelsResponse);
-  rpc setAuths(SetAuthsRequest)
-    returns (VisibilityLabelsResponse);
-  rpc clearAuths(SetAuthsRequest)
-    returns (VisibilityLabelsResponse);
-  rpc getAuths(GetAuthsRequest)
-    returns (GetAuthsResponse);
-  rpc listLabels(ListLabelsRequest)
-    returns (ListLabelsResponse);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/WAL.proto b/hbase-native-client/if/WAL.proto
deleted file mode 100644
index 2061b22..0000000
--- a/hbase-native-client/if/WAL.proto
+++ /dev/null
@@ -1,173 +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.
- */
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "WALProtos";
-option java_generic_services = false;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "Client.proto";
-
-message WALHeader {
-  optional bool has_compression = 1;
-  optional bytes encryption_key = 2;
-  optional bool has_tag_compression = 3;
-  optional string writer_cls_name = 4;
-  optional string cell_codec_cls_name = 5;
-}
-
-/*
- * Protocol buffer version of WALKey; see WALKey comment, not really a key but WALEdit header
- * for some KVs
- */
-message WALKey {
-  required bytes encoded_region_name = 1;
-  required bytes table_name = 2;
-  required uint64 log_sequence_number = 3;
-  required uint64 write_time = 4;
-  /*
-  This parameter is deprecated in favor of clusters which
-  contains the list of clusters that have consumed the change.
-  It is retained so that the log created by earlier releases (0.94)
-  can be read by the newer releases.
-  */
-  optional UUID cluster_id = 5 [deprecated=true];
-
-  repeated FamilyScope scopes = 6;
-  optional uint32 following_kv_count = 7;
-
-  /*
-  This field contains the list of clusters that have
-  consumed the change
-  */
-  repeated UUID cluster_ids = 8;
-
-  optional uint64 nonceGroup = 9;
-  optional uint64 nonce = 10;
-  optional uint64 orig_sequence_number = 11;
-
-/*
-  optional CustomEntryType custom_entry_type = 9;
-
-  enum CustomEntryType {
-    COMPACTION = 0;
-  }
-*/
-}
-
-enum ScopeType {
-  REPLICATION_SCOPE_LOCAL = 0;
-  REPLICATION_SCOPE_GLOBAL = 1;
-}
-
-message FamilyScope {
-  required bytes family = 1;
-  required ScopeType scope_type = 2;
-}
-
-/**
- * Custom WAL entries
- */
-
-/**
- * Special WAL entry to hold all related to a compaction.
- * Written to WAL before completing compaction.  There is
- * sufficient info in the below message to complete later
- * the * compaction should we fail the WAL write.
- */
-message CompactionDescriptor {
-  required bytes table_name = 1; // TODO: WALKey already stores these, might remove
-  required bytes encoded_region_name = 2;
-  required bytes family_name = 3;
-  repeated string compaction_input = 4; // relative to store dir
-  repeated string compaction_output = 5;
-  required string store_home_dir = 6; // relative to region dir
-  optional bytes  region_name = 7; // full region name
-}
-
-/**
- * Special WAL entry to hold all related to a flush.
- */
-message FlushDescriptor {
-  enum FlushAction {
-    START_FLUSH = 0;
-    COMMIT_FLUSH = 1;
-    ABORT_FLUSH = 2;
-    CANNOT_FLUSH = 3; // marker for indicating that a flush has been requested but cannot complete
-  }
-
-  message StoreFlushDescriptor {
-    required bytes family_name = 1;
-    required string store_home_dir = 2; //relative to region dir
-    repeated string flush_output = 3; // relative to store dir (if this is a COMMIT_FLUSH)
-  }
-
-  required FlushAction action = 1;
-  required bytes table_name = 2;
-  required bytes encoded_region_name = 3;
-  optional uint64 flush_sequence_number = 4;
-  repeated StoreFlushDescriptor store_flushes = 5;
-  optional bytes  region_name = 6; // full region name
-}
-
-message StoreDescriptor {
-  required bytes family_name = 1;
-  required string store_home_dir = 2; //relative to region dir
-  repeated string store_file = 3; // relative to store dir
-  optional uint64 store_file_size_bytes = 4; // size of store file
-}
-
-/**
- * Special WAL entry used for writing bulk load events to WAL
- */
-message BulkLoadDescriptor {
-  required TableName table_name = 1;
-  required bytes encoded_region_name = 2;
-  repeated StoreDescriptor stores = 3;
-  required int64 bulkload_seq_num = 4;
-}
-
-/**
- * Special WAL entry to hold all related to a region event (open/close).
- */
-message RegionEventDescriptor {
-  enum EventType {
-    REGION_OPEN = 0;
-    REGION_CLOSE = 1;
-  }
-
-  required EventType event_type = 1;
-  required bytes table_name = 2;
-  required bytes encoded_region_name = 3;
-  optional uint64 log_sequence_number = 4;
-  repeated StoreDescriptor stores = 5;
-  optional ServerName server = 6;  // Server who opened the region
-  optional bytes  region_name = 7; // full region name
-}
-
-/**
- * A trailer that is appended to the end of a properly closed WAL file.
- * If missing, this is either a legacy or a corrupted WAL file.
- * N.B. This trailer currently doesn't contain any information and we
- * purposefully don't expose it in the WAL APIs. It's for future growth.
- */
-message WALTrailer {
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/ZooKeeper.proto b/hbase-native-client/if/ZooKeeper.proto
deleted file mode 100644
index 41c0e0e..0000000
--- a/hbase-native-client/if/ZooKeeper.proto
+++ /dev/null
@@ -1,176 +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.
- */
-
-// ZNode data in hbase are serialized protobufs with a four byte
-// 'magic' 'PBUF' prefix.
-package hbase.pb;
-
-option java_package = "org.apache.hadoop.hbase.protobuf.generated";
-option java_outer_classname = "ZooKeeperProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-option optimize_for = SPEED;
-
-import "HBase.proto";
-import "ClusterStatus.proto";
-
-/**
- * Content of the meta-region-server znode.
- */
-message MetaRegionServer {
-  // The ServerName hosting the meta region currently, or destination server,
-  // if meta region is in transition.
-  required ServerName server = 1;
-  // The major version of the rpc the server speaks.  This is used so that
-  // clients connecting to the cluster can have prior knowledge of what version
-  // to send to a RegionServer.  AsyncHBase will use this to detect versions.
-  optional uint32 rpc_version = 2;
-
-  // State of the region transition. OPEN means fully operational 'hbase:meta'
-  optional RegionState.State state = 3;
-}
-
-/**
- * Content of the master znode.
- */
-message Master {
-  // The ServerName of the current Master
-  required ServerName master = 1;
-  // Major RPC version so that clients can know what version the master can accept.
-  optional uint32 rpc_version = 2;
-  optional uint32 info_port = 3;
-}
-
-/**
- * Content of the '/hbase/running', cluster state, znode.
- */
-message ClusterUp {
-  // If this znode is present, cluster is up.  Currently
-  // the data is cluster start_date.
-  required string start_date = 1;
-}
-
-/**
- * WAL SplitLog directory znodes have this for content.  Used doing distributed
- * WAL splitting.  Holds current state and name of server that originated split.
- */
-message SplitLogTask {
-  enum State {
-    UNASSIGNED = 0;
-    OWNED = 1;
-    RESIGNED = 2;
-    DONE = 3;
-    ERR = 4;
-  }
-  enum RecoveryMode {
-    UNKNOWN = 0;
-    LOG_SPLITTING = 1;
-    LOG_REPLAY = 2;
-  }
-  required State state = 1;
-  required ServerName server_name = 2;
-  optional RecoveryMode mode = 3 [default = UNKNOWN];
-}
-
-/**
- * The znode that holds state of table.
- * Deprected, table state is stored in table descriptor on HDFS.
- */
-message DeprecatedTableState {
-  // Table's current state
-  enum State {
-    ENABLED = 0;
-    DISABLED = 1;
-    DISABLING = 2;
-    ENABLING = 3;
-  }
-  // This is the table's state.  If no znode for a table,
-  // its state is presumed enabled.  See o.a.h.h.zookeeper.ZKTable class
-  // for more.
-  required State state = 1 [default = ENABLED];
-}
-
-message TableCF {
-  optional TableName table_name = 1;
-  repeated bytes families = 2;
-}
-
-/**
- * Used by replication. Holds a replication peer key.
- */
-message ReplicationPeer {
-  // clusterkey is the concatenation of the slave cluster's
-  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-  required string clusterkey = 1;
-  optional string replicationEndpointImpl = 2;
-  repeated BytesBytesPair data = 3;
-  repeated NameStringPair configuration = 4;
-  repeated TableCF table_cfs = 5;
-}
-
-/**
- * Used by replication. Holds whether enabled or disabled
- */
-message ReplicationState {
-  enum State {
-    ENABLED = 0;
-    DISABLED = 1;
-  }
-  required State state = 1;
-}
-
-/**
- * Used by replication. Holds the current position in an WAL file.
- */
-message ReplicationHLogPosition {
-  required int64 position = 1;
-}
-
-/**
- * Used by replication. Used to lock a region server during failover.
- */
-message ReplicationLock {
-  required string lock_owner = 1;
-}
-
-/**
- * Metadata associated with a table lock in zookeeper
- */
-message TableLock {
-  optional TableName table_name = 1;
-  optional ServerName lock_owner = 2;
-  optional int64 thread_id = 3;
-  optional bool is_shared = 4;
-  optional string purpose = 5;
-  optional int64 create_time = 6;
-}
-
-/**
- * State of the switch.
- */
-message SwitchState {
-  optional bool enabled = 1;
-}
-
-/**
- * State for split and merge, used in hbck
- */
-message SplitAndMergeState {
-  optional bool split_enabled = 1;
-  optional bool merge_enabled = 2;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/test.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/test.proto b/hbase-native-client/if/test.proto
deleted file mode 100644
index 72b68e9..0000000
--- a/hbase-native-client/if/test.proto
+++ /dev/null
@@ -1,43 +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.
- */
-
-option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
-option java_outer_classname = "TestProtos";
-option java_generate_equals_and_hash = true;
-
-message EmptyRequestProto {
-}
-
-message EmptyResponseProto {
-}
-
-message EchoRequestProto {
-  required string message = 1;
-}
-
-message EchoResponseProto {
-  required string message = 1;
-}
-
-message PauseRequestProto {
-  required uint32 ms = 1;
-}
-
-message AddrResponseProto {
-  required string addr = 1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/if/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/if/test_rpc_service.proto b/hbase-native-client/if/test_rpc_service.proto
deleted file mode 100644
index 2730403..0000000
--- a/hbase-native-client/if/test_rpc_service.proto
+++ /dev/null
@@ -1,36 +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.
- */
-option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
-option java_outer_classname = "TestRpcServiceProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "test.proto";
-
-
-/**
- * A protobuf service for use in tests
- */
-service TestProtobufRpcProto {
-  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
-  rpc echo(EchoRequestProto) returns (EchoResponseProto);
-  rpc error(EmptyRequestProto) returns (EmptyResponseProto);
-  rpc pause(PauseRequestProto) returns (EmptyResponseProto);
-  rpc addr(EmptyRequestProto) returns (AddrResponseProto);
-  rpc socketNotOpen(EmptyRequestProto) returns (EmptyResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/BUCK b/hbase-native-client/include/hbase/client/BUCK
new file mode 100644
index 0000000..66d6896
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/BUCK
@@ -0,0 +1,93 @@
+##
+# 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.
+
+# This is the main library.
+cxx_library(
+    name="client",
+    header_namespace="hbase/client",
+    exported_headers=[
+        "async-client-scanner.h",
+        "async-connection.h",
+        "async-region-locator.h",
+        "async-rpc-retrying-caller-factory.h",
+        "async-rpc-retrying-caller.h",
+        "async-table-result-scanner.h",
+        "client.h",
+        "cell.h",
+        "filter.h",
+        "query.h",
+        "keyvalue-codec.h",
+        "region-location.h",
+        "location-cache.h",
+        "connection-configuration.h",
+        # TODO: move this out of exported
+        # Once meta lookup works
+        "meta-utils.h",
+        "get.h",
+        "increment.h",
+        "mutation.h",
+        "put.h",
+        "delete.h",
+        "scan.h",
+        "append.h",
+        "result.h",
+        "result-scanner.h",
+        "request-converter.h",
+        "response-converter.h",
+        "table.h",
+        "async-scan-rpc-retrying-caller.h",
+        "raw-async-table.h",
+        "raw-scan-result-consumer.h",
+        "scan-result-cache.h",
+        "hbase-rpc-controller.h",
+        "time-range.h",
+        "zk-util.h",
+        "action.h",
+        "multi-response.h",
+        "region-request.h",
+        "region-result.h",
+        "row.h",
+        "server-request.h",
+        "async-batch-rpc-retrying-caller.h",
+    ],
+    deps=[
+        "//include/hbase/exceptions:exceptions",
+        "//include/hbase/utils:utils",
+        "//include/hbase/connection:connection",
+        "//include/hbase/client:conf",
+        "//src/hbase/if:if",
+        "//include/hbase/serde:serde",
+        "//third-party:folly",
+        "//third-party:wangle",
+        "//third-party:zookeeper_mt",
+    ],
+    compiler_flags=['-Weffc++', '-ggdb'],
+    visibility=[
+        'PUBLIC',
+    ],)
+cxx_library(
+    name="conf",
+    header_namespace="hbase/client",
+    exported_headers=[
+        "configuration.h",
+        "hbase-configuration-loader.h",
+    ],
+    deps=["//src/hbase/utils:utils", "//third-party:folly"],
+    compiler_flags=['-Weffc++', '-ggdb'],
+    visibility=[
+        'PUBLIC',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/action.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/action.h b/hbase-native-client/include/hbase/client/action.h
new file mode 100644
index 0000000..2288f12
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/action.h
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include "hbase/client/row.h"
+
+namespace hbase {
+class Action {
+ public:
+  Action(std::shared_ptr<hbase::Row> action, int32_t original_index)
+      : action_(action), original_index_(original_index) {}
+  ~Action() {}
+
+  int32_t original_index() const { return original_index_; }
+
+  std::shared_ptr<hbase::Row> action() const { return action_; }
+
+ private:
+  std::shared_ptr<hbase::Row> action_;
+  int32_t original_index_;
+  int64_t nonce_ = -1;
+  int32_t replica_id_ = -1;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/append.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/append.h b/hbase-native-client/include/hbase/client/append.h
new file mode 100644
index 0000000..e7f9a6d
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/append.h
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Append : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Append(const std::string& row) : Mutation(row) {}
+  Append(const Append& cappend) : Mutation(cappend) {}
+  Append& operator=(const Append& cappend) {
+    Mutation::operator=(cappend);
+    return *this;
+  }
+
+  ~Append() = default;
+
+  /**
+   *  @brief Add the specified column and value to this Append operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param value value to append
+   */
+  Append& Add(const std::string& family, const std::string& qualifier, const std::string& value);
+  Append& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-batch-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-batch-rpc-retrying-caller.h b/hbase-native-client/include/hbase/client/async-batch-rpc-retrying-caller.h
new file mode 100644
index 0000000..39b7aa9
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-batch-rpc-retrying-caller.h
@@ -0,0 +1,185 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/Format.h>
+#include <folly/Try.h>
+#include <folly/futures/Future.h>
+#include <folly/futures/Promise.h>
+#include <folly/io/IOBuf.h>
+#include <folly/io/async/HHWheelTimer.h>
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+
+#include <algorithm>
+#include <chrono>
+#include <functional>
+#include <map>
+#include <memory>
+#include <mutex>
+#include <stdexcept>
+#include <string>
+#include <tuple>
+#include <type_traits>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/action.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/location-cache.h"
+#include "hbase/client/multi-response.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/region-request.h"
+#include "hbase/client/region-result.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/client/result.h"
+#include "hbase/client/row.h"
+#include "hbase/client/server-request.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/security/user.h"
+#include "hbase/utils/connection-util.h"
+#include "hbase/utils/sys-util.h"
+#include "hbase/utils/time-util.h"
+
+namespace hbase {
+/* Equals function for ServerName */
+struct ServerNameEquals {
+  bool operator()(const std::shared_ptr<pb::ServerName> &lhs,
+                  const std::shared_ptr<pb::ServerName> &rhs) const {
+    return (lhs->start_code() == rhs->start_code() && lhs->host_name() == rhs->host_name() &&
+            lhs->port() == rhs->port());
+  }
+};
+
+struct ServerNameHash {
+  /** hash */
+  std::size_t operator()(const std::shared_ptr<pb::ServerName> &sn) const {
+    std::size_t h = 0;
+    boost::hash_combine(h, sn->start_code());
+    boost::hash_combine(h, sn->host_name());
+    boost::hash_combine(h, sn->port());
+    return h;
+  }
+};
+
+template <typename REQ, typename RESP>
+class AsyncBatchRpcRetryingCaller {
+ public:
+  using ActionsByServer =
+      std::unordered_map<std::shared_ptr<pb::ServerName>, std::shared_ptr<ServerRequest>,
+                         ServerNameHash, ServerNameEquals>;
+  using ActionsByRegion = ServerRequest::ActionsByRegion;
+
+  AsyncBatchRpcRetryingCaller(std::shared_ptr<AsyncConnection> conn,
+                              std::shared_ptr<folly::HHWheelTimer> retry_timer,
+                              std::shared_ptr<pb::TableName> table_name,
+                              const std::vector<REQ> &actions, std::chrono::nanoseconds pause_ns,
+                              int32_t max_attempts, std::chrono::nanoseconds operation_timeout_ns,
+                              std::chrono::nanoseconds rpc_timeout_ns,
+                              int32_t start_log_errors_count);
+
+  ~AsyncBatchRpcRetryingCaller();
+
+  folly::Future<std::vector<folly::Try<RESP>>> Call();
+
+ private:
+  int64_t RemainingTimeNs();
+
+  void LogException(int32_t tries, std::shared_ptr<RegionRequest> region_request,
+                    const folly::exception_wrapper &ew,
+                    std::shared_ptr<pb::ServerName> server_name);
+
+  void LogException(int32_t tries,
+                    const std::vector<std::shared_ptr<RegionRequest>> &region_requests,
+                    const folly::exception_wrapper &ew,
+                    std::shared_ptr<pb::ServerName> server_name);
+
+  const std::string GetExtraContextForError(std::shared_ptr<pb::ServerName> server_name);
+
+  void AddError(const std::shared_ptr<Action> &action, const folly::exception_wrapper &ew,
+                std::shared_ptr<pb::ServerName> server_name);
+
+  void AddError(const std::vector<std::shared_ptr<Action>> &actions,
+                const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
+
+  void FailOne(const std::shared_ptr<Action> &action, int32_t tries,
+               const folly::exception_wrapper &ew, int64_t current_time, const std::string extras);
+
+  void FailAll(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries,
+               const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
+
+  void FailAll(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
+
+  void AddAction2Error(uint64_t action_index, const ThrowableWithExtraContext &twec);
+
+  void OnError(const ActionsByRegion &actions_by_region, int32_t tries,
+               const folly::exception_wrapper &ew, std::shared_ptr<pb::ServerName> server_name);
+
+  void TryResubmit(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
+
+  folly::Future<std::vector<folly::Try<std::shared_ptr<RegionLocation>>>> GetRegionLocations(
+      const std::vector<std::shared_ptr<Action>> &actions, int64_t locate_timeout_ns);
+
+  void GroupAndSend(const std::vector<std::shared_ptr<Action>> &actions, int32_t tries);
+
+  folly::Future<std::vector<folly::Try<std::unique_ptr<Response>>>> GetMultiResponse(
+      const ActionsByServer &actions_by_server);
+
+  void Send(const ActionsByServer &actions_by_server, int32_t tries);
+
+  void OnComplete(const ActionsByRegion &actions_by_region, int32_t tries,
+                  const std::shared_ptr<pb::ServerName> server_name,
+                  const std::unique_ptr<MultiResponse> multi_results);
+
+  void OnComplete(const std::shared_ptr<Action> &action,
+                  const std::shared_ptr<RegionRequest> &region_request, int32_t tries,
+                  const std::shared_ptr<pb::ServerName> &server_name,
+                  const std::shared_ptr<RegionResult> &region_result,
+                  std::vector<std::shared_ptr<Action>> &failed_actions);
+
+ private:
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<hbase::AsyncConnection> conn_;
+  std::shared_ptr<pb::TableName> table_name_;
+  std::vector<std::shared_ptr<Action>> actions_;
+  std::chrono::nanoseconds pause_ns_;
+  int32_t max_attempts_ = 0;
+  std::chrono::nanoseconds operation_timeout_ns_;
+  std::chrono::nanoseconds rpc_timeout_ns_;
+  int32_t start_log_errors_count_ = 0;
+
+  int64_t start_ns_ = TimeUtil::GetNowNanos();
+  int32_t tries_ = 1;
+  std::map<uint64_t, folly::Promise<RESP>> action2promises_;
+  std::vector<folly::Future<RESP>> action2futures_;
+  std::map<uint64_t, std::shared_ptr<std::vector<ThrowableWithExtraContext>>> action2errors_;
+
+  std::shared_ptr<AsyncRegionLocator> location_cache_ = nullptr;
+  std::shared_ptr<RpcClient> rpc_client_ = nullptr;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_pool_ = nullptr;
+
+  std::recursive_mutex multi_mutex_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-client-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-client-scanner.h b/hbase-native-client/include/hbase/client/async-client-scanner.h
new file mode 100644
index 0000000..cccf50b
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-client-scanner.h
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Format.h>
+#include <folly/Logging.h>
+#include <folly/futures/Future.h>
+#include <folly/io/async/EventBase.h>
+#include <folly/io/async/HHWheelTimer.h>
+
+#include <algorithm>
+#include <chrono>
+#include <functional>
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/async-rpc-retrying-caller-factory.h"
+#include "hbase/client/async-rpc-retrying-caller.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/raw-scan-result-consumer.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/client/result.h"
+#include "hbase/client/scan-result-cache.h"
+#include "hbase/client/scan.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/utils/connection-util.h"
+#include "hbase/utils/sys-util.h"
+#include "hbase/utils/time-util.h"
+
+using std::chrono::nanoseconds;
+using std::chrono::milliseconds;
+
+namespace hbase {
+class OpenScannerResponse {
+ public:
+  OpenScannerResponse(std::shared_ptr<hbase::RpcClient> rpc_client,
+                      const std::unique_ptr<Response>& resp,
+                      std::shared_ptr<RegionLocation> region_location,
+                      std::shared_ptr<hbase::HBaseRpcController> controller)
+      : rpc_client_(rpc_client), region_location_(region_location), controller_(controller) {
+    scan_resp_ = std::static_pointer_cast<pb::ScanResponse>(resp->resp_msg());
+    cell_scanner_ = resp->cell_scanner();
+  }
+  std::shared_ptr<hbase::RpcClient> rpc_client_;
+  std::shared_ptr<pb::ScanResponse> scan_resp_;
+  std::shared_ptr<RegionLocation> region_location_;
+  std::shared_ptr<hbase::HBaseRpcController> controller_;
+  std::shared_ptr<CellScanner> cell_scanner_;
+};
+
+class AsyncClientScanner : public std::enable_shared_from_this<AsyncClientScanner> {
+ public:
+  template <typename... T>
+  static std::shared_ptr<AsyncClientScanner> Create(T&&... all) {
+    return std::shared_ptr<AsyncClientScanner>(new AsyncClientScanner(std::forward<T>(all)...));
+  }
+
+  void Start();
+
+ private:
+  // methods
+  AsyncClientScanner(std::shared_ptr<AsyncConnection> conn, std::shared_ptr<Scan> scan,
+                     std::shared_ptr<pb::TableName> table_name,
+                     std::shared_ptr<RawScanResultConsumer> consumer, nanoseconds pause,
+                     uint32_t max_retries, nanoseconds scan_timeout_nanos,
+                     nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
+
+  folly::Future<std::shared_ptr<OpenScannerResponse>> CallOpenScanner(
+      std::shared_ptr<hbase::RpcClient> rpc_client,
+      std::shared_ptr<hbase::HBaseRpcController> controller,
+      std::shared_ptr<hbase::RegionLocation> loc);
+
+  void OpenScanner();
+
+  void StartScan(std::shared_ptr<OpenScannerResponse> resp);
+
+  RegionLocateType GetLocateType(const Scan& scan);
+
+ private:
+  // data
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<Scan> scan_;
+  std::shared_ptr<pb::TableName> table_name_;
+  std::shared_ptr<ScanResultCache> results_cache_;
+  std::shared_ptr<RawScanResultConsumer> consumer_;
+  nanoseconds pause_;
+  uint32_t max_retries_;
+  nanoseconds scan_timeout_nanos_;
+  nanoseconds rpc_timeout_nanos_;
+  uint32_t start_log_errors_count_;
+  uint32_t max_attempts_;
+  uint32_t open_scanner_tries_ = 0;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-connection.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-connection.h b/hbase-native-client/include/hbase/client/async-connection.h
new file mode 100644
index 0000000..547ac76
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-connection.h
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/futures/Future.h>
+#include <folly/io/IOBuf.h>
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-region-locator.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/connection-configuration.h"
+#include "hbase/client/hbase-configuration-loader.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/keyvalue-codec.h"
+#include "hbase/client/location-cache.h"
+#include "hbase/if/Cell.pb.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+class AsyncRpcRetryingCallerFactory;
+
+class AsyncConnection {
+ public:
+  AsyncConnection() {}
+  virtual ~AsyncConnection() {}
+  virtual std::shared_ptr<Configuration> conf() = 0;
+  virtual std::shared_ptr<ConnectionConfiguration> connection_conf() = 0;
+  virtual std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() = 0;
+  virtual std::shared_ptr<RpcClient> rpc_client() = 0;
+  virtual std::shared_ptr<AsyncRegionLocator> region_locator() = 0;
+  virtual std::shared_ptr<HBaseRpcController> CreateRpcController() = 0;
+  virtual std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() = 0;
+  virtual std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() = 0;
+  virtual std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() = 0;
+  virtual void Close() = 0;
+};
+
+class AsyncConnectionImpl : public AsyncConnection,
+                            public std::enable_shared_from_this<AsyncConnectionImpl> {
+ public:
+  virtual ~AsyncConnectionImpl();
+
+  // See https://mortoray.com/2013/08/02/safely-using-enable_shared_from_this/
+  template <typename... T>
+  static std::shared_ptr<AsyncConnectionImpl> Create(T&&... all) {
+    auto conn =
+        std::shared_ptr<AsyncConnectionImpl>(new AsyncConnectionImpl(std::forward<T>(all)...));
+    conn->Init();
+    return conn;
+  }
+
+  std::shared_ptr<Configuration> conf() override { return conf_; }
+  std::shared_ptr<ConnectionConfiguration> connection_conf() override { return connection_conf_; }
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory() override {
+    return caller_factory_;
+  }
+  std::shared_ptr<RpcClient> rpc_client() override { return rpc_client_; }
+  std::shared_ptr<LocationCache> location_cache() { return location_cache_; }
+  std::shared_ptr<AsyncRegionLocator> region_locator() override { return location_cache_; }
+  std::shared_ptr<HBaseRpcController> CreateRpcController() override {
+    return std::make_shared<HBaseRpcController>();
+  }
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() override { return cpu_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() override { return io_executor_; }
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor() override {
+    return retry_executor_;
+  }
+
+  void Close() override;
+
+ protected:
+  AsyncConnectionImpl() {}
+
+ private:
+  /** Parameter name for HBase client IO thread pool size. Defaults to num cpus */
+  static constexpr const char* kClientIoThreadPoolSize = "hbase.client.io.thread.pool.size";
+  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
+  static constexpr const char* kClientCpuThreadPoolSize = "hbase.client.cpu.thread.pool.size";
+  /** The RPC codec to encode cells. For now it is KeyValueCodec */
+  static constexpr const char* kRpcCodec = "hbase.client.rpc.codec";
+
+  std::shared_ptr<Configuration> conf_;
+  std::shared_ptr<ConnectionConfiguration> connection_conf_;
+  std::shared_ptr<AsyncRpcRetryingCallerFactory> caller_factory_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> retry_executor_;
+  std::shared_ptr<LocationCache> location_cache_;
+  std::shared_ptr<RpcClient> rpc_client_;
+  bool is_closed_ = false;
+
+ private:
+  explicit AsyncConnectionImpl(std::shared_ptr<Configuration> conf) : conf_(conf) {}
+  void Init();
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-region-locator.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-region-locator.h b/hbase-native-client/include/hbase/client/async-region-locator.h
new file mode 100644
index 0000000..d42038e
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-region-locator.h
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/futures/Future.h>
+#include <memory>
+#include <string>
+
+#include "hbase/client/region-location.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/serde/region-info.h"
+#include "hbase/serde/server-name.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+class AsyncRegionLocator {
+ public:
+  AsyncRegionLocator() {}
+  virtual ~AsyncRegionLocator() = default;
+
+  /**
+   * The only method clients should use for meta lookups. If corresponding
+   * location is cached, it's returned from the cache, otherwise lookup
+   * in meta table is done, location is cached and then returned.
+   * It's expected that tiny fraction of invocations incurs meta scan.
+   * This method is to look up non-meta regions; use LocateMeta() to get the
+   * location of hbase:meta region.
+   *
+   * @param tn Table name of the table to look up. This object must live until
+   * after the future is returned
+   *
+   * @param row of the table to look up. This object must live until after the
+   * future is returned
+   */
+  virtual folly::Future<std::shared_ptr<RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) = 0;
+  /**
+   * Update cached region location, possibly using the information from exception.
+   */
+  virtual void UpdateCachedLocation(const RegionLocation &loc,
+                                    const folly::exception_wrapper &error) = 0;
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-rpc-retrying-caller-factory.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-rpc-retrying-caller-factory.h b/hbase-native-client/include/hbase/client/async-rpc-retrying-caller-factory.h
new file mode 100644
index 0000000..a580896
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-rpc-retrying-caller-factory.h
@@ -0,0 +1,343 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Logging.h>
+#include <folly/io/async/EventBase.h>
+#include <chrono>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-batch-rpc-retrying-caller.h"
+#include "hbase/client/async-rpc-retrying-caller.h"
+#include "hbase/client/async-scan-rpc-retrying-caller.h"
+#include "hbase/client/raw-scan-result-consumer.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/row.h"
+#include "hbase/client/scan-result-cache.h"
+#include "hbase/client/scan.h"
+
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+class AsyncConnection;
+
+template <typename RESP>
+class SingleRequestCallerBuilder
+    : public std::enable_shared_from_this<SingleRequestCallerBuilder<RESP>> {
+ public:
+  explicit SingleRequestCallerBuilder(std::shared_ptr<AsyncConnection> conn,
+                                      std::shared_ptr<folly::HHWheelTimer> retry_timer)
+      : conn_(conn),
+        retry_timer_(retry_timer),
+        table_name_(nullptr),
+        rpc_timeout_nanos_(conn->connection_conf()->rpc_timeout()),
+        pause_(conn->connection_conf()->pause()),
+        operation_timeout_nanos_(conn->connection_conf()->operation_timeout()),
+        max_retries_(conn->connection_conf()->max_retries()),
+        start_log_errors_count_(conn->connection_conf()->start_log_errors_count()),
+        locate_type_(RegionLocateType::kCurrent) {}
+
+  virtual ~SingleRequestCallerBuilder() = default;
+
+  typedef SingleRequestCallerBuilder<RESP> GenericThisType;
+  typedef std::shared_ptr<GenericThisType> SharedThisPtr;
+
+  SharedThisPtr table(std::shared_ptr<pb::TableName> table_name) {
+    table_name_ = table_name;
+    return shared_this();
+  }
+
+  SharedThisPtr rpc_timeout(std::chrono::nanoseconds rpc_timeout_nanos) {
+    rpc_timeout_nanos_ = rpc_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr operation_timeout(std::chrono::nanoseconds operation_timeout_nanos) {
+    operation_timeout_nanos_ = operation_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr pause(std::chrono::nanoseconds pause) {
+    pause_ = pause;
+    return shared_this();
+  }
+
+  SharedThisPtr max_retries(uint32_t max_retries) {
+    max_retries_ = max_retries;
+    return shared_this();
+  }
+
+  SharedThisPtr start_log_errors_count(uint32_t start_log_errors_count) {
+    start_log_errors_count_ = start_log_errors_count;
+    return shared_this();
+  }
+
+  SharedThisPtr row(const std::string& row) {
+    row_ = row;
+    return shared_this();
+  }
+
+  SharedThisPtr locate_type(RegionLocateType locate_type) {
+    locate_type_ = locate_type;
+    return shared_this();
+  }
+
+  SharedThisPtr action(Callable<RESP> callable) {
+    callable_ = callable;
+    return shared_this();
+  }
+
+  folly::Future<RESP> Call() { return Build()->Call(); }
+
+  std::shared_ptr<AsyncSingleRequestRpcRetryingCaller<RESP>> Build() {
+    return std::make_shared<AsyncSingleRequestRpcRetryingCaller<RESP>>(
+        conn_, retry_timer_, table_name_, row_, locate_type_, callable_, pause_, max_retries_,
+        operation_timeout_nanos_, rpc_timeout_nanos_, start_log_errors_count_);
+  }
+
+ private:
+  SharedThisPtr shared_this() {
+    return std::enable_shared_from_this<GenericThisType>::shared_from_this();
+  }
+
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<pb::TableName> table_name_;
+  std::chrono::nanoseconds rpc_timeout_nanos_;
+  std::chrono::nanoseconds operation_timeout_nanos_;
+  std::chrono::nanoseconds pause_;
+  uint32_t max_retries_;
+  uint32_t start_log_errors_count_;
+  std::string row_;
+  RegionLocateType locate_type_;
+  Callable<RESP> callable_;
+};  // end of SingleRequestCallerBuilder
+
+template <typename REQ, typename RESP>
+class BatchCallerBuilder : public std::enable_shared_from_this<BatchCallerBuilder<REQ, RESP>> {
+ public:
+  explicit BatchCallerBuilder(std::shared_ptr<AsyncConnection> conn,
+                              std::shared_ptr<folly::HHWheelTimer> retry_timer)
+      : conn_(conn), retry_timer_(retry_timer) {}
+
+  virtual ~BatchCallerBuilder() = default;
+
+  typedef std::shared_ptr<BatchCallerBuilder<REQ, RESP>> SharedThisPtr;
+
+  SharedThisPtr table(std::shared_ptr<pb::TableName> table_name) {
+    table_name_ = table_name;
+    return shared_this();
+  }
+
+  SharedThisPtr actions(std::shared_ptr<std::vector<REQ>> actions) {
+    actions_ = actions;
+    return shared_this();
+  }
+
+  SharedThisPtr operation_timeout(std::chrono::nanoseconds operation_timeout_nanos) {
+    operation_timeout_nanos_ = operation_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr rpc_timeout(std::chrono::nanoseconds rpc_timeout_nanos) {
+    rpc_timeout_nanos_ = rpc_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr pause(std::chrono::nanoseconds pause_ns) {
+    pause_ns_ = pause_ns;
+    return shared_this();
+  }
+
+  SharedThisPtr max_attempts(int32_t max_attempts) {
+    max_attempts_ = max_attempts;
+    return shared_this();
+  }
+
+  SharedThisPtr start_log_errors_count(int32_t start_log_errors_count) {
+    start_log_errors_count_ = start_log_errors_count;
+    return shared_this();
+  }
+
+  folly::Future<std::vector<folly::Try<RESP>>> Call() { return Build()->Call(); }
+
+  std::shared_ptr<AsyncBatchRpcRetryingCaller<REQ, RESP>> Build() {
+    return std::make_shared<AsyncBatchRpcRetryingCaller<REQ, RESP>>(
+        conn_, retry_timer_, table_name_, *actions_, pause_ns_, max_attempts_,
+        operation_timeout_nanos_, rpc_timeout_nanos_, start_log_errors_count_);
+  }
+
+ private:
+  SharedThisPtr shared_this() {
+    return std::enable_shared_from_this<BatchCallerBuilder>::shared_from_this();
+  }
+
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<hbase::pb::TableName> table_name_ = nullptr;
+  std::shared_ptr<std::vector<REQ>> actions_ = nullptr;
+  std::chrono::nanoseconds pause_ns_;
+  int32_t max_attempts_ = 0;
+  std::chrono::nanoseconds operation_timeout_nanos_;
+  std::chrono::nanoseconds rpc_timeout_nanos_;
+  int32_t start_log_errors_count_ = 0;
+};
+
+class ScanCallerBuilder : public std::enable_shared_from_this<ScanCallerBuilder> {
+ public:
+  explicit ScanCallerBuilder(std::shared_ptr<AsyncConnection> conn,
+                             std::shared_ptr<folly::HHWheelTimer> retry_timer)
+      : conn_(conn),
+        retry_timer_(retry_timer),
+        rpc_timeout_nanos_(conn->connection_conf()->rpc_timeout()),
+        pause_(conn->connection_conf()->pause()),
+        scan_timeout_nanos_(conn->connection_conf()->scan_timeout()),
+        max_retries_(conn->connection_conf()->max_retries()),
+        start_log_errors_count_(conn->connection_conf()->start_log_errors_count()),
+        scanner_id_(-1) {}
+
+  virtual ~ScanCallerBuilder() = default;
+
+  typedef ScanCallerBuilder GenericThisType;
+  typedef std::shared_ptr<ScanCallerBuilder> SharedThisPtr;
+
+  SharedThisPtr rpc_client(std::shared_ptr<hbase::RpcClient> rpc_client) {
+    rpc_client_ = rpc_client;
+    return shared_this();
+  }
+
+  SharedThisPtr rpc_timeout(nanoseconds rpc_timeout_nanos) {
+    rpc_timeout_nanos_ = rpc_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr scan_timeout(nanoseconds scan_timeout_nanos) {
+    scan_timeout_nanos_ = scan_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr scanner_lease_timeout(nanoseconds scanner_lease_timeout_nanos) {
+    scanner_lease_timeout_nanos_ = scanner_lease_timeout_nanos;
+    return shared_this();
+  }
+
+  SharedThisPtr pause(nanoseconds pause) {
+    pause_ = pause;
+    return shared_this();
+  }
+
+  SharedThisPtr max_retries(uint32_t max_retries) {
+    max_retries_ = max_retries;
+    return shared_this();
+  }
+
+  SharedThisPtr start_log_errors_count(uint32_t start_log_errors_count) {
+    start_log_errors_count_ = start_log_errors_count;
+    return shared_this();
+  }
+
+  SharedThisPtr region_location(std::shared_ptr<RegionLocation> region_location) {
+    region_location_ = region_location;
+    return shared_this();
+  }
+
+  SharedThisPtr scanner_id(int64_t scanner_id) {
+    scanner_id_ = scanner_id;
+    return shared_this();
+  }
+
+  SharedThisPtr scan(std::shared_ptr<Scan> scan) {
+    scan_ = scan;
+    return shared_this();
+  }
+
+  SharedThisPtr results_cache(std::shared_ptr<ScanResultCache> results_cache) {
+    results_cache_ = results_cache;
+    return shared_this();
+  }
+
+  SharedThisPtr consumer(std::shared_ptr<RawScanResultConsumer> consumer) {
+    consumer_ = consumer;
+    return shared_this();
+  }
+
+  std::shared_ptr<AsyncScanRpcRetryingCaller> Build() {
+    return std::make_shared<AsyncScanRpcRetryingCaller>(
+        conn_, retry_timer_, rpc_client_, scan_, scanner_id_, results_cache_, consumer_,
+        region_location_, scanner_lease_timeout_nanos_, pause_, max_retries_, scan_timeout_nanos_,
+        rpc_timeout_nanos_, start_log_errors_count_);
+  }
+
+ private:
+  SharedThisPtr shared_this() {
+    return std::enable_shared_from_this<GenericThisType>::shared_from_this();
+  }
+
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<hbase::RpcClient> rpc_client_;
+  std::shared_ptr<Scan> scan_;
+  nanoseconds rpc_timeout_nanos_;
+  nanoseconds scan_timeout_nanos_;
+  nanoseconds scanner_lease_timeout_nanos_;
+  nanoseconds pause_;
+  uint32_t max_retries_;
+  uint32_t start_log_errors_count_;
+  std::shared_ptr<RegionLocation> region_location_;
+  int64_t scanner_id_;
+  std::shared_ptr<RawScanResultConsumer> consumer_;
+  std::shared_ptr<ScanResultCache> results_cache_;
+};  // end of ScanCallerBuilder
+
+class AsyncRpcRetryingCallerFactory {
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+
+ public:
+  explicit AsyncRpcRetryingCallerFactory(std::shared_ptr<AsyncConnection> conn,
+                                         std::shared_ptr<folly::HHWheelTimer> retry_timer)
+      : conn_(conn), retry_timer_(retry_timer) {}
+
+  virtual ~AsyncRpcRetryingCallerFactory() = default;
+
+  template <typename RESP>
+  std::shared_ptr<SingleRequestCallerBuilder<RESP>> Single() {
+    return std::make_shared<SingleRequestCallerBuilder<RESP>>(conn_, retry_timer_);
+  }
+
+  template <typename REQ, typename RESP>
+  std::shared_ptr<BatchCallerBuilder<REQ, RESP>> Batch() {
+    return std::make_shared<BatchCallerBuilder<REQ, RESP>>(conn_, retry_timer_);
+  }
+
+  std::shared_ptr<ScanCallerBuilder> Scan() {
+    return std::make_shared<ScanCallerBuilder>(conn_, retry_timer_);
+  }
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-rpc-retrying-caller.h b/hbase-native-client/include/hbase/client/async-rpc-retrying-caller.h
new file mode 100644
index 0000000..b5f4ad2
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-rpc-retrying-caller.h
@@ -0,0 +1,115 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/futures/Future.h>
+#include <folly/io/async/EventBase.h>
+#include <folly/io/async/HHWheelTimer.h>
+
+#include <algorithm>
+#include <chrono>
+#include <functional>
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+#include "hbase/client/async-connection.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/region-location.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+template <typename T>
+using Supplier = std::function<T()>;
+
+template <typename T>
+using Consumer = std::function<void(T)>;
+
+template <typename R, typename S, typename... I>
+using ReqConverter = std::function<R(const S&, const I&...)>;
+
+template <typename R, typename S>
+using RespConverter = std::function<R(const S&)>;
+
+template <typename RESP>
+using RpcCallback = std::function<void(const RESP&)>;
+
+template <typename REQ, typename RESP>
+using RpcCall = std::function<folly::Future<std::unique_ptr<RESP>>(
+    std::shared_ptr<RpcClient>, std::shared_ptr<RegionLocation>,
+    std::shared_ptr<HBaseRpcController>, std::unique_ptr<REQ>)>;
+
+template <typename RESP>
+using Callable =
+    std::function<folly::Future<RESP>(std::shared_ptr<HBaseRpcController>,
+                                      std::shared_ptr<RegionLocation>, std::shared_ptr<RpcClient>)>;
+
+template <typename RESP>
+class AsyncSingleRequestRpcRetryingCaller {
+ public:
+  AsyncSingleRequestRpcRetryingCaller(
+      std::shared_ptr<AsyncConnection> conn, std::shared_ptr<folly::HHWheelTimer> retry_timer,
+      std::shared_ptr<hbase::pb::TableName> table_name, const std::string& row,
+      RegionLocateType locate_type, Callable<RESP> callable, std::chrono::nanoseconds pause,
+      uint32_t max_retries, std::chrono::nanoseconds operation_timeout_nanos,
+      std::chrono::nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
+
+  virtual ~AsyncSingleRequestRpcRetryingCaller();
+
+  folly::Future<RESP> Call();
+
+ private:
+  void LocateThenCall();
+
+  void OnError(const folly::exception_wrapper& error, Supplier<std::string> err_msg,
+               Consumer<folly::exception_wrapper> update_cached_location);
+
+  void Call(const RegionLocation& loc);
+
+  void CompleteExceptionally();
+
+  int64_t RemainingTimeNs();
+
+  static void ResetController(std::shared_ptr<HBaseRpcController> controller,
+                              const int64_t& timeout_ns);
+
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<hbase::pb::TableName> table_name_;
+  std::string row_;
+  RegionLocateType locate_type_;
+  Callable<RESP> callable_;
+  std::chrono::nanoseconds pause_;
+  uint32_t max_retries_;
+  std::chrono::nanoseconds operation_timeout_nanos_;
+  std::chrono::nanoseconds rpc_timeout_nanos_;
+  uint32_t start_log_errors_count_;
+  std::shared_ptr<folly::Promise<RESP>> promise_;
+  std::shared_ptr<HBaseRpcController> controller_;
+  uint64_t start_ns_;
+  uint32_t tries_;
+  std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions_;
+  uint32_t max_attempts_;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-scan-rpc-retrying-caller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-scan-rpc-retrying-caller.h b/hbase-native-client/include/hbase/client/async-scan-rpc-retrying-caller.h
new file mode 100644
index 0000000..68382e6
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-scan-rpc-retrying-caller.h
@@ -0,0 +1,233 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Conv.h>
+#include <folly/ExceptionWrapper.h>
+#include <folly/Format.h>
+#include <folly/Logging.h>
+#include <folly/futures/Future.h>
+#include <folly/io/async/EventBase.h>
+#include <folly/io/async/HHWheelTimer.h>
+
+#include <algorithm>
+#include <chrono>
+#include <functional>
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/hbase-rpc-controller.h"
+#include "hbase/client/raw-scan-result-consumer.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/client/result.h"
+#include "hbase/client/scan-result-cache.h"
+#include "hbase/client/scan.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/utils/bytes-util.h"
+#include "hbase/utils/connection-util.h"
+#include "hbase/utils/optional.h"
+#include "hbase/utils/sys-util.h"
+#include "hbase/utils/time-util.h"
+
+using std::chrono::nanoseconds;
+using std::chrono::milliseconds;
+
+namespace hbase {
+
+class AsyncScanRpcRetryingCaller;
+
+// The resume method is allowed to be called in another thread so here we also use the
+// ResumerState to prevent race. The initial state is INITIALIZED, and in most cases, when back
+// from onNext or onHeartbeat, we will call the prepare method to change the state to SUSPENDED,
+// and when user calls resume method, we will change the state to RESUMED. But the resume method
+// could be called in other thread, and in fact, user could just do this:
+// controller.suspend().resume()
+// This is strange but valid. This means the scan could be resumed before we call the prepare
+// method to do the actual suspend work. So in the resume method, we will check if the state is
+// INTIALIZED, if it is, then we will just set the state to RESUMED and return. And in prepare
+// method, if the state is RESUMED already, we will just return an let the scan go on.
+// Notice that, the public methods of this class is supposed to be called by upper layer only, and
+// package private methods can only be called within the implementation of
+// AsyncScanSingleRegionRpcRetryingCaller.
+// TODO: Unlike the Java counter part, we do not do scan lease renewals in a background thread.
+// Since there is also no async scan API exposed to the users, only ScanResultConsumer is the
+// AsyncTableResultScanner which will only pause the scanner if the result cache is maxed. The
+// application is expected to consume the scan results before the scanner lease timeout.
+class ScanResumerImpl : public ScanResumer {
+ public:
+  explicit ScanResumerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller);
+
+  virtual ~ScanResumerImpl() = default;
+
+  /**
+   * Resume the scan. You are free to call it multiple time but only the first call will take
+   * effect.
+   */
+  void Resume() override;
+
+  // return false if the scan has already been resumed. See the comment above for ScanResumerImpl
+  // for more details.
+  bool Prepare(std::shared_ptr<pb::ScanResponse> resp, int num_complete_rows);
+
+ private:
+  // INITIALIZED -> SUSPENDED -> RESUMED
+  // INITIALIZED -> RESUMED
+  ScanResumerState state_ = ScanResumerState::kInitialized;
+  std::mutex mutex_;
+  std::shared_ptr<pb::ScanResponse> resp_ = nullptr;
+  int64_t num_complete_rows_ = 0;
+  std::shared_ptr<AsyncScanRpcRetryingCaller> caller_;
+};
+
+class ScanControllerImpl : public ScanController {
+ public:
+  virtual ~ScanControllerImpl() = default;
+
+  explicit ScanControllerImpl(std::shared_ptr<AsyncScanRpcRetryingCaller> caller);
+
+  /**
+   * Suspend the scan.
+   * <p>
+   * This means we will stop fetching data in background, i.e., will not call onNext any more
+   * before you resume the scan.
+   * @return A resumer used to resume the scan later.
+   */
+  std::shared_ptr<ScanResumer> Suspend();
+
+  /**
+   * Terminate the scan.
+   * <p>
+   * This is useful when you have got enough results and want to stop the scan in onNext method,
+   * or you want to stop the scan in onHeartbeat method because it has spent too many time.
+   */
+  void Terminate();
+
+  // return the current state, and set the state to DESTROYED.
+  ScanControllerState Destroy();
+
+  std::shared_ptr<ScanResumerImpl> resumer() { return resumer_; }
+
+ private:
+  void PreCheck();
+
+  std::string DebugString(ScanControllerState state);
+
+  std::string DebugString(ScanResumerState state);
+
+ private:
+  // Make sure the methods are only called in this thread.
+  std::thread::id caller_thread_id_ = std::this_thread::get_id();
+  // INITIALIZED -> SUSPENDED -> DESTROYED
+  // INITIALIZED -> TERMINATED -> DESTROYED
+  // INITIALIZED -> DESTROYED
+  // If the state is incorrect we will throw IllegalStateException.
+  ScanControllerState state_ = ScanControllerState::kInitialized;
+  std::shared_ptr<ScanResumerImpl> resumer_ = nullptr;
+  std::shared_ptr<AsyncScanRpcRetryingCaller> caller_;
+};
+
+class AsyncScanRpcRetryingCaller : public std::enable_shared_from_this<AsyncScanRpcRetryingCaller> {
+ public:
+  AsyncScanRpcRetryingCaller(std::shared_ptr<AsyncConnection> conn,
+                             std::shared_ptr<folly::HHWheelTimer> retry_timer,
+                             std::shared_ptr<hbase::RpcClient> rpc_client,
+                             std::shared_ptr<Scan> scan, int64_t scanner_id,
+                             std::shared_ptr<ScanResultCache> results_cache,
+                             std::shared_ptr<RawScanResultConsumer> consumer,
+                             std::shared_ptr<RegionLocation> region_location,
+                             nanoseconds scanner_lease_timeout_nanos, nanoseconds pause,
+                             uint32_t max_retries, nanoseconds scan_timeout_nanos,
+                             nanoseconds rpc_timeout_nanos, uint32_t start_log_errors_count);
+
+  folly::Future<bool> Start(std::shared_ptr<HBaseRpcController> controller,
+                            std::shared_ptr<pb::ScanResponse> open_scan_resp,
+                            const std::shared_ptr<CellScanner> cell_scanner);
+
+ private:
+  int64_t RemainingTimeNs();
+  void OnComplete(std::shared_ptr<HBaseRpcController> controller,
+                  std::shared_ptr<pb::ScanResponse> resp,
+                  const std::shared_ptr<CellScanner> cell_scanner);
+
+  void CompleteOrNext(std::shared_ptr<pb::ScanResponse> resp);
+
+  void CompleteExceptionally(bool close_scanner);
+
+  void CompleteNoMoreResults();
+
+  void CompleteWhenNoMoreResultsInRegion();
+
+  void CompleteWithNextStartRow(std::string row, bool inclusive);
+
+  void UpdateNextStartRowWhenError(const Result& result);
+
+  void CompleteWhenError(bool close_scanner);
+
+  void OnError(const folly::exception_wrapper& e);
+
+  bool NoMoreResultsForScan(const Scan& scan, const pb::RegionInfo& info);
+
+  void Next();
+
+  void Call();
+
+  void CloseScanner();
+
+  void ResetController(std::shared_ptr<HBaseRpcController> controller,
+                       const int64_t& timeout_nanos);
+
+ private:
+  std::shared_ptr<AsyncConnection> conn_;
+  std::shared_ptr<folly::HHWheelTimer> retry_timer_;
+  std::shared_ptr<hbase::RpcClient> rpc_client_;
+  std::shared_ptr<Scan> scan_;
+  int64_t scanner_id_;
+  std::shared_ptr<ScanResultCache> results_cache_;
+  std::shared_ptr<RawScanResultConsumer> consumer_;
+  std::shared_ptr<RegionLocation> region_location_;
+  nanoseconds scanner_lease_timeout_nanos_;
+  nanoseconds pause_;
+  uint32_t max_retries_;
+  nanoseconds scan_timeout_nanos_;
+  nanoseconds rpc_timeout_nanos_;
+  uint32_t start_log_errors_count_;
+  std::shared_ptr<folly::Promise<bool>> promise_;
+  std::shared_ptr<HBaseRpcController> controller_;
+  optional<std::string> next_start_row_when_error_ = optional<std::string>();
+  bool include_next_start_row_when_error_ = true;
+  uint64_t start_ns_;
+  uint32_t tries_;
+  std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions_;
+  uint32_t max_attempts_;
+  int64_t next_call_seq_ = -1L;
+
+  friend class ScanResumerImpl;
+  friend class ScanControllerImpl;
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/async-table-result-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/async-table-result-scanner.h b/hbase-native-client/include/hbase/client/async-table-result-scanner.h
new file mode 100644
index 0000000..0e1d444
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/async-table-result-scanner.h
@@ -0,0 +1,98 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Conv.h>
+#include <folly/ExceptionWrapper.h>
+#include <folly/Logging.h>
+#include <chrono>
+#include <condition_variable>
+#include <memory>
+#include <mutex>
+#include <queue>
+#include <string>
+#include <vector>
+
+#include "hbase/client/raw-scan-result-consumer.h"
+#include "hbase/client/result-scanner.h"
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+class AsyncTableResultScanner : public ResultScanner, public RawScanResultConsumer {
+ public:
+  explicit AsyncTableResultScanner(int64_t max_cache_size);
+
+  virtual ~AsyncTableResultScanner();
+
+  void Close() override;
+
+  std::shared_ptr<Result> Next() override;
+
+  void OnNext(const std::vector<std::shared_ptr<Result>> &results,
+              std::shared_ptr<ScanController> controller) override;
+
+  /**
+   * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
+   * onNext.
+   * <p>
+   * This method give you a chance to terminate a slow scan operation.
+   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
+   *          instance is only valid within the scope of onHeartbeat method. You can only call its
+   *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
+   */
+  void OnHeartbeat(std::shared_ptr<ScanController> controller) override;
+
+  /**
+   * Indicate that we hit an unrecoverable error and the scan operation is terminated.
+   * <p>
+   * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
+   */
+  void OnError(const folly::exception_wrapper &error) override;
+
+  /**
+   * Indicate that the scan operation is completed normally.
+   */
+  void OnComplete() override;
+
+  // For testing
+  uint32_t num_prefetch_stopped() { return num_prefetch_stopped_; }
+
+ private:
+  void AddToCache(const std::vector<std::shared_ptr<Result>> &results);
+
+  template <typename T>
+  inline size_t EstimatedSizeWithSharedPtr(std::shared_ptr<T> t);
+
+  void StopPrefetch(std::shared_ptr<ScanController> controller);
+
+ private:
+  std::queue<std::shared_ptr<Result>> queue_;
+  std::mutex mutex_;
+  std::condition_variable cond_;
+  folly::exception_wrapper error_;
+  int64_t cache_size_;
+  int64_t max_cache_size_;
+  bool closed_;
+  std::shared_ptr<ScanResumer> resumer_ = nullptr;
+  uint32_t num_prefetch_stopped_ = 0;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/cell.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/cell.h b/hbase-native-client/include/hbase/client/cell.h
new file mode 100644
index 0000000..7a62a9b
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/cell.h
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <string>
+
+namespace hbase {
+
+enum class CellType {
+  MINIMUM = 0,
+  PUT = 4,
+  DELETE = 8,
+  DELETE_FAMILY_VERSION = 10,
+  DELETE_COLUMN = 12,
+  DELETE_FAMILY = 14,
+  MAXIMUM = 255
+};
+
+class Cell {
+ public:
+  Cell(const std::string &row, const std::string &family, const std::string &qualifier,
+       const int64_t timestamp, const std::string &value, const hbase::CellType &cell_type);
+  Cell(const Cell &cell);
+  Cell &operator=(const Cell &cell);
+  virtual ~Cell();
+  const std::string &Row() const;
+  const std::string &Family() const;
+  const std::string &Qualifier() const;
+  int64_t Timestamp() const;
+  const std::string &Value() const;
+  CellType Type() const;
+  int64_t SequenceId() const;
+  std::string DebugString() const;
+  /** Returns estimated size of the Cell object including deep heap space usage
+    * of its data. Notice that this is a very rough estimate. */
+  size_t EstimatedSize() const;
+
+ private:
+  std::string row_;
+  std::string family_;
+  std::string qualifier_;
+  // Since java does not have unsigned, we are also using signed numerics here
+  // so that we won't have surprises when large uint64's are treated as
+  // negative values in the java server side
+  int64_t timestamp_;
+  hbase::CellType cell_type_;
+  std::string value_;
+  int64_t sequence_id_;
+
+ private:
+  static const char *TypeToString(CellType type);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/client.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/client.h b/hbase-native-client/include/hbase/client/client.h
new file mode 100644
index 0000000..2e77df9
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/client.h
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/configuration.h"
+
+#include "hbase/client/table.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+class Table;
+/**
+ * Client.
+ *
+ * This is the class that provides access to an HBase cluster.
+ * It is thread safe and does connection pooling. Current recommendations are to
+ * have only one Client per cluster around.
+ */
+class Client {
+ public:
+  /**
+   * @brief Create a new client.
+   * @param quorum_spec Where to connect to get Zookeeper bootstrap information.
+   */
+  Client();
+  explicit Client(const Configuration& conf);
+  ~Client() = default;
+
+  /**
+   * @brief Retrieve a Table implementation for accessing a table.
+   * @param - table_name
+   */
+  std::unique_ptr<::hbase::Table> Table(const pb::TableName& table_name);
+
+  /**
+   * @brief Close the Client connection.
+   */
+  void Close();
+
+  /**
+   * @brief Internal. DO NOT USE.
+   */
+  std::shared_ptr<AsyncConnectionImpl> async_connection() { return async_connection_; }
+
+ private:
+  /** Data */
+  std::shared_ptr<AsyncConnectionImpl> async_connection_;
+
+ private:
+  /** Methods */
+  void Init(const Configuration& conf);
+};
+
+}  // namespace hbase


[05/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/scan.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/scan.cc b/hbase-native-client/src/hbase/client/scan.cc
new file mode 100644
index 0000000..0cadcf0
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/scan.cc
@@ -0,0 +1,186 @@
+/*
+ * 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 "hbase/client/scan.h"
+
+#include <algorithm>
+#include <iterator>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+
+Scan::Scan() {}
+
+Scan::~Scan() {}
+
+Scan::Scan(const std::string &start_row) : start_row_(start_row) { CheckRow(start_row_); }
+
+Scan::Scan(const std::string &start_row, const std::string &stop_row)
+    : start_row_(start_row), stop_row_(stop_row) {
+  CheckRow(start_row_);
+  CheckRow(stop_row_);
+}
+
+Scan::Scan(const Scan &scan) : Query(scan) {
+  start_row_ = scan.start_row_;
+  stop_row_ = scan.stop_row_;
+  max_versions_ = scan.max_versions_;
+  caching_ = scan.caching_;
+  max_result_size_ = scan.max_result_size_;
+  cache_blocks_ = scan.cache_blocks_;
+  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
+  reversed_ = scan.reversed_;
+  allow_partial_results_ = scan.allow_partial_results_;
+  consistency_ = scan.consistency_;
+  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
+  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
+}
+
+Scan &Scan::operator=(const Scan &scan) {
+  Query::operator=(scan);
+  start_row_ = scan.start_row_;
+  stop_row_ = scan.stop_row_;
+  max_versions_ = scan.max_versions_;
+  caching_ = scan.caching_;
+  max_result_size_ = scan.max_result_size_;
+  cache_blocks_ = scan.cache_blocks_;
+  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
+  reversed_ = scan.reversed_;
+  allow_partial_results_ = scan.allow_partial_results_;
+  consistency_ = scan.consistency_;
+  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
+  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
+  return *this;
+}
+
+Scan::Scan(const Get &get) {
+  cache_blocks_ = get.CacheBlocks();
+  max_versions_ = get.MaxVersions();
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.FamilyMap().begin(), get.FamilyMap().end());
+}
+
+Scan &Scan::AddFamily(const std::string &family) {
+  const auto &it = family_map_.find(family);
+  /**
+   * Check if any qualifiers are already present or not.
+   * Remove all existing qualifiers if the given family is already present in
+   * the map
+   */
+  if (family_map_.end() != it) {
+    it->second.clear();
+  } else {
+    family_map_[family];
+  }
+  return *this;
+}
+
+Scan &Scan::AddColumn(const std::string &family, const std::string &qualifier) {
+  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
+  /**
+   * Check if any qualifiers are already present or not.
+   * Add only if qualifiers for a given family are not present
+   */
+  if (it == family_map_[family].end()) {
+    family_map_[family].push_back(qualifier);
+  }
+  return *this;
+}
+
+void Scan::SetReversed(bool reversed) { reversed_ = reversed; }
+
+bool Scan::IsReversed() const { return reversed_; }
+
+void Scan::SetStartRow(const std::string &start_row) { start_row_ = start_row; }
+
+const std::string &Scan::StartRow() const { return start_row_; }
+
+void Scan::SetStopRow(const std::string &stop_row) { stop_row_ = stop_row; }
+
+const std::string &Scan::StopRow() const { return stop_row_; }
+
+void Scan::SetCaching(int caching) { caching_ = caching; }
+
+int Scan::Caching() const { return caching_; }
+
+Scan &Scan::SetConsistency(const hbase::pb::Consistency consistency) {
+  consistency_ = consistency;
+  return *this;
+}
+
+hbase::pb::Consistency Scan::Consistency() const { return consistency_; }
+
+void Scan::SetCacheBlocks(bool cache_blocks) { cache_blocks_ = cache_blocks; }
+
+bool Scan::CacheBlocks() const { return cache_blocks_; }
+
+void Scan::SetAllowPartialResults(bool allow_partial_results) {
+  allow_partial_results_ = allow_partial_results;
+}
+
+bool Scan::AllowPartialResults() const { return allow_partial_results_; }
+
+void Scan::SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand) {
+  load_column_families_on_demand_ = load_column_families_on_demand;
+}
+
+bool Scan::LoadColumnFamiliesOnDemand() const { return load_column_families_on_demand_; }
+
+Scan &Scan::SetMaxVersions(uint32_t max_versions) {
+  max_versions_ = max_versions;
+  return *this;
+}
+
+int Scan::MaxVersions() const { return max_versions_; }
+
+void Scan::SetMaxResultSize(int64_t max_result_size) { max_result_size_ = max_result_size; }
+
+int64_t Scan::MaxResultSize() const { return max_result_size_; }
+
+Scan &Scan::SetTimeRange(int64_t min_stamp, int64_t max_stamp) {
+  tr_.reset(new TimeRange(min_stamp, max_stamp));
+  return *this;
+}
+
+Scan &Scan::SetTimeStamp(int64_t timestamp) {
+  tr_.reset(new TimeRange(timestamp, timestamp + 1));
+  return *this;
+}
+
+const TimeRange &Scan::Timerange() const { return *tr_; }
+
+void Scan::CheckRow(const std::string &row) {
+  const int32_t kMaxRowLength = std::numeric_limits<int16_t>::max();
+  int row_length = row.size();
+  if (0 == row_length) {
+    throw std::runtime_error("Row length can't be 0");
+  }
+  if (row_length > kMaxRowLength) {
+    throw std::runtime_error("Length of " + row + " is greater than max row size: " +
+                             std::to_string(kMaxRowLength));
+  }
+}
+
+bool Scan::HasFamilies() const { return !family_map_.empty(); }
+
+const std::map<std::string, std::vector<std::string>> &Scan::FamilyMap() const {
+  return family_map_;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/scanner-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/scanner-test.cc b/hbase-native-client/src/hbase/client/scanner-test.cc
new file mode 100644
index 0000000..8a58e34
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/scanner-test.cc
@@ -0,0 +1,368 @@
+/*
+ * 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 <folly/Conv.h>
+#include <gtest/gtest.h>
+
+#include <chrono>
+#include <thread>
+#include <vector>
+
+#include "hbase/client/async-client-scanner.h"
+#include "hbase/client/async-table-result-scanner.h"
+#include "hbase/client/cell.h"
+#include "hbase/client/client.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/filter.h"
+#include "hbase/client/get.h"
+#include "hbase/client/hbase-configuration-loader.h"
+#include "hbase/client/put.h"
+#include "hbase/client/result.h"
+#include "hbase/client/row.h"
+#include "hbase/client/table.h"
+#include "hbase/if/Comparator.pb.h"
+#include "hbase/if/Filter.pb.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/test-util/test-util.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Cell;
+using hbase::ComparatorFactory;
+using hbase::Comparator;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::Put;
+using hbase::Result;
+using hbase::Scan;
+using hbase::Table;
+using hbase::TestUtil;
+using hbase::TimeUtil;
+using hbase::AsyncClientScanner;
+using hbase::AsyncTableResultScanner;
+using hbase::FilterFactory;
+using hbase::pb::CompareType;
+
+class ScannerTest : public ::testing::Test {
+ public:
+  static std::unique_ptr<hbase::TestUtil> test_util;
+  static const uint32_t num_rows;
+
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util = std::make_unique<hbase::TestUtil>();
+    test_util->StartMiniCluster(2);
+  }
+};
+std::unique_ptr<hbase::TestUtil> ScannerTest::test_util = nullptr;
+const uint32_t ScannerTest::num_rows = 1000;
+
+std::string Family(uint32_t i) { return "f" + folly::to<std::string>(i); }
+
+std::string Row(uint32_t i, int width) {
+  std::ostringstream s;
+  s.fill('0');
+  s.width(width);
+  s << i;
+  return "row" + s.str();
+}
+
+std::string Row(uint32_t i) { return Row(i, 3); }
+
+std::unique_ptr<Put> MakePut(const std::string &row, uint32_t num_families) {
+  auto put = std::make_unique<Put>(row);
+
+  for (uint32_t i = 0; i < num_families; i++) {
+    put->AddColumn(Family(i), "q1", row);
+    put->AddColumn(Family(i), "q2", row + "-" + row);
+  }
+
+  return std::move(put);
+}
+
+void CheckResult(const Result &r, std::string expected_row, uint32_t num_families) {
+  VLOG(1) << r.DebugString();
+  auto row = r.Row();
+  ASSERT_EQ(row, expected_row);
+  ASSERT_EQ(r.Cells().size(), num_families * 2);
+  for (uint32_t i = 0; i < num_families; i++) {
+    ASSERT_EQ(*r.Value(Family(i), "q1"), row);
+    ASSERT_EQ(*r.Value(Family(i), "q2"), row + "-" + row);
+  }
+}
+
+void CreateTable(std::string table_name, uint32_t num_families, uint32_t num_rows,
+                 int32_t num_regions) {
+  LOG(INFO) << "Creating the table " << table_name
+            << " with num_regions:" << folly::to<std::string>(num_regions);
+  std::vector<std::string> families;
+  for (uint32_t i = 0; i < num_families; i++) {
+    families.push_back(Family(i));
+  }
+  if (num_regions <= 1) {
+    ScannerTest::test_util->CreateTable(table_name, families);
+  } else {
+    std::vector<std::string> keys;
+    for (int32_t i = 0; i < num_regions - 1; i++) {
+      keys.push_back(Row(i * (num_rows / (num_regions - 1))));
+      LOG(INFO) << "Split key:" << keys[keys.size() - 1];
+    }
+    ScannerTest::test_util->CreateTable(table_name, families, keys);
+  }
+}
+
+std::unique_ptr<hbase::Client> CreateTableAndWriteData(std::string table_name,
+                                                       uint32_t num_families, uint32_t num_rows,
+                                                       int32_t num_regions) {
+  CreateTable(table_name, num_families, num_rows, num_regions);
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+  auto client = std::make_unique<hbase::Client>(*ScannerTest::test_util->conf());
+  auto table = client->Table(tn);
+
+  LOG(INFO) << "Writing data to the table, num_rows:" << num_rows;
+  // Perform Puts
+  for (uint32_t i = 0; i < num_rows; i++) {
+    table->Put(*MakePut(Row(i), num_families));
+  }
+  return std::move(client);
+}
+
+void TestScan(const Scan &scan, uint32_t num_families, int32_t start, int32_t num_rows,
+              Table *table) {
+  LOG(INFO) << "Starting scan for the test with start:" << scan.StartRow()
+            << ", stop:" << scan.StopRow() << " expected_num_rows:" << num_rows;
+  auto scanner = table->Scan(scan);
+
+  uint32_t i = start;
+  auto r = scanner->Next();
+  while (r != nullptr) {
+    CheckResult(*r, Row(i++), num_families);
+    r = scanner->Next();
+  }
+  ASSERT_EQ(i - start, num_rows);
+}
+
+void TestScan(const Scan &scan, int32_t start, int32_t num_rows, Table *table) {
+  TestScan(scan, 1, start, num_rows, table);
+}
+
+void TestScan(uint32_t num_families, int32_t start, int32_t stop, int32_t num_rows, Table *table) {
+  Scan scan{};
+  if (start >= 0) {
+    scan.SetStartRow(Row(start));
+  } else {
+    start = 0;  // neded for below logic
+  }
+  if (stop >= 0) {
+    scan.SetStopRow(Row(stop));
+  }
+
+  TestScan(scan, num_families, start, num_rows, table);
+}
+
+void TestScan(int32_t start, int32_t stop, int32_t num_rows, Table *table) {
+  TestScan(1, start, stop, num_rows, table);
+}
+
+void TestScan(uint32_t num_families, std::string start, std::string stop, int32_t num_rows,
+              Table *table) {
+  Scan scan{};
+
+  scan.SetStartRow(start);
+  scan.SetStopRow(stop);
+
+  LOG(INFO) << "Starting scan for the test with start:" << start << ", stop:" << stop
+            << " expected_num_rows:" << num_rows;
+  auto scanner = table->Scan(scan);
+
+  uint32_t i = 0;
+  auto r = scanner->Next();
+  while (r != nullptr) {
+    VLOG(1) << r->DebugString();
+    i++;
+    ASSERT_EQ(r->Map().size(), num_families);
+    r = scanner->Next();
+  }
+  ASSERT_EQ(i, num_rows);
+}
+
+void TestScan(std::string start, std::string stop, int32_t num_rows, Table *table) {
+  TestScan(1, start, stop, num_rows, table);
+}
+
+void TestScanCombinations(Table *table, uint32_t num_families) {
+  // full table
+  TestScan(num_families, -1, -1, 1000, table);
+  TestScan(num_families, -1, 999, 999, table);
+  TestScan(num_families, 0, -1, 1000, table);
+  TestScan(num_families, 0, 999, 999, table);
+  TestScan(num_families, 10, 990, 980, table);
+  TestScan(num_families, 1, 998, 997, table);
+
+  TestScan(num_families, 123, 345, 222, table);
+  TestScan(num_families, 234, 456, 222, table);
+  TestScan(num_families, 345, 567, 222, table);
+  TestScan(num_families, 456, 678, 222, table);
+
+  // single results
+  TestScan(num_families, 111, 111, 1, table);  // split keys are like 111, 222, 333, etc
+  TestScan(num_families, 111, 112, 1, table);
+  TestScan(num_families, 332, 332, 1, table);
+  TestScan(num_families, 332, 333, 1, table);
+  TestScan(num_families, 333, 333, 1, table);
+  TestScan(num_families, 333, 334, 1, table);
+  TestScan(num_families, 42, 42, 1, table);
+  TestScan(num_families, 921, 921, 1, table);
+  TestScan(num_families, 0, 0, 1, table);
+  TestScan(num_families, 0, 1, 1, table);
+  TestScan(num_families, 999, 999, 1, table);
+
+  // few results
+  TestScan(num_families, 0, 0, 1, table);
+  TestScan(num_families, 0, 2, 2, table);
+  TestScan(num_families, 0, 5, 5, table);
+  TestScan(num_families, 10, 15, 5, table);
+  TestScan(num_families, 105, 115, 10, table);
+  TestScan(num_families, 111, 221, 110, table);
+  TestScan(num_families, 111, 222, 111, table);  // crossing region boundary 111-222
+  TestScan(num_families, 111, 223, 112, table);
+  TestScan(num_families, 111, 224, 113, table);
+  TestScan(num_families, 990, 999, 9, table);
+  TestScan(num_families, 900, 998, 98, table);
+
+  // empty results
+  TestScan(num_families, "a", "a", 0, table);
+  TestScan(num_families, "a", "r", 0, table);
+  TestScan(num_families, "", "r", 0, table);
+  TestScan(num_families, "s", "", 0, table);
+  TestScan(num_families, "s", "z", 0, table);
+  TestScan(num_families, Row(110) + "a", Row(111), 0, table);
+  TestScan(num_families, Row(111) + "a", Row(112), 0, table);
+  TestScan(num_families, Row(123) + "a", Row(124), 0, table);
+
+  // custom
+  TestScan(num_families, Row(111, 3), Row(1111, 4), 1, table);
+  TestScan(num_families, Row(0, 3), Row(0, 4), 1, table);
+  TestScan(num_families, Row(999, 3), Row(9999, 4), 1, table);
+  TestScan(num_families, Row(111, 3), Row(1111, 4), 1, table);
+  TestScan(num_families, Row(0, 3), Row(9999, 4), 1000, table);
+  TestScan(num_families, "a", "z", 1000, table);
+}
+
+// some of these tests are from TestAsyncTableScan* and some from TestFromClientSide* and
+// TestScannersFromClientSide*
+
+TEST_F(ScannerTest, SingleRegionScan) {
+  auto client = CreateTableAndWriteData("t_single_region_scan", 1, num_rows, 1);
+  auto table = client->Table(folly::to<hbase::pb::TableName>("t_single_region_scan"));
+
+  TestScanCombinations(table.get(), 1);
+}
+
+TEST_F(ScannerTest, MultiRegionScan) {
+  auto client = CreateTableAndWriteData("t_multi_region_scan", 1, num_rows, 10);
+  auto table = client->Table(folly::to<hbase::pb::TableName>("t_multi_region_scan"));
+
+  TestScanCombinations(table.get(), 1);
+}
+
+TEST_F(ScannerTest, ScanWithPauses) {
+  auto max_result_size =
+      ScannerTest::test_util->conf()->GetInt("hbase.client.scanner.max.result.size", 2097152);
+  ScannerTest::test_util->conf()->SetInt("hbase.client.scanner.max.result.size", 100);
+  auto client = CreateTableAndWriteData("t_multi_region_scan", 1, num_rows, 5);
+  auto table = client->Table(folly::to<hbase::pb::TableName>("t_multi_region_scan"));
+
+  VLOG(1) << "Starting scan for the test";
+  Scan scan{};
+  scan.SetCaching(100);
+  auto scanner = table->Scan(scan);
+
+  uint32_t i = 0;
+  auto r = scanner->Next();
+  while (r != nullptr) {
+    CheckResult(*r, Row(i++), 1);
+    r = scanner->Next();
+    std::this_thread::sleep_for(TimeUtil::MillisToNanos(10));
+  }
+
+  auto s = static_cast<AsyncTableResultScanner *>(scanner.get());
+  ASSERT_GT(s->num_prefetch_stopped(), 0);
+
+  ASSERT_EQ(i, num_rows);
+  ScannerTest::test_util->conf()->SetInt("hbase.client.scanner.max.result.size", max_result_size);
+}
+
+TEST_F(ScannerTest, ScanWithFilters) {
+  auto client = CreateTableAndWriteData("t_scan_with_filters", 1, num_rows, 1);
+  auto table = client->Table(folly::to<hbase::pb::TableName>("t_scan_with_filters"));
+
+  Scan scan{};
+  scan.SetFilter(FilterFactory::ValueFilter(CompareType::GREATER_OR_EQUAL,
+                                            *ComparatorFactory::BinaryComparator(Row(800))));
+
+  TestScan(scan, 800, 200, table.get());
+}
+
+TEST_F(ScannerTest, ScanMultiFamily) {
+  auto client = CreateTableAndWriteData("t_scan_multi_family", 3, num_rows, 1);
+  auto table = client->Table(folly::to<hbase::pb::TableName>("t_scan_multi_family"));
+
+  TestScanCombinations(table.get(), 3);
+}
+
+TEST_F(ScannerTest, ScanNullQualifier) {
+  std::string table_name{"t_scan_null_qualifier"};
+  std::string row{"row"};
+  CreateTable(table_name, 1, 1, 1);
+
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+  auto client = std::make_unique<hbase::Client>(*ScannerTest::test_util->conf());
+  auto table = client->Table(tn);
+
+  // Perform Puts
+  Put put{row};
+  put.AddColumn(Family(0), "q1", row);
+  put.AddColumn(Family(0), "", row);
+  table->Put(put);
+
+  Scan scan1{};
+  scan1.AddColumn(Family(0), "");
+  auto scanner1 = table->Scan(scan1);
+  auto r1 = scanner1->Next();
+  ASSERT_EQ(r1->Cells().size(), 1);
+  ASSERT_EQ(scanner1->Next(), nullptr);
+
+  Scan scan2{};
+  scan2.AddFamily(Family(0));
+  auto scanner2 = table->Scan(scan2);
+  auto r2 = scanner2->Next();
+  ASSERT_EQ(r2->Cells().size(), 2);
+  ASSERT_EQ(scanner2->Next(), nullptr);
+}
+
+TEST_F(ScannerTest, ScanNoResults) {
+  std::string table_name{"t_scan_no_results"};
+  auto client = CreateTableAndWriteData(table_name, 1, num_rows, 3);
+  auto table = client->Table(folly::to<hbase::pb::TableName>(table_name));
+
+  Scan scan{};
+  scan.AddColumn(Family(0), "non_existing_qualifier");
+
+  TestScan(scan, 0, 0, table.get());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/simple-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/simple-client.cc b/hbase-native-client/src/hbase/client/simple-client.cc
new file mode 100644
index 0000000..25f203a
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/simple-client.cc
@@ -0,0 +1,207 @@
+/*
+ * 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 <folly/Logging.h>
+#include <folly/Random.h>
+#include <gflags/gflags.h>
+
+#include <atomic>
+#include <chrono>
+#include <iostream>
+#include <thread>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/client.h"
+#include "hbase/client/get.h"
+#include "hbase/client/hbase-configuration-loader.h"
+#include "hbase/client/put.h"
+#include "hbase/client/scan.h"
+#include "hbase/client/table.h"
+#include "hbase/serde/server-name.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Client;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::HBaseConfigurationLoader;
+using hbase::Scan;
+using hbase::Put;
+using hbase::Result;
+using hbase::Table;
+using hbase::pb::TableName;
+using hbase::pb::ServerName;
+using hbase::TimeUtil;
+
+DEFINE_string(table, "test_table", "What table to do the reads or writes");
+DEFINE_string(row, "row_", "row prefix");
+DEFINE_string(zookeeper, "localhost:2181", "What zk quorum to talk to");
+DEFINE_string(conf, "", "Conf directory to read the config from (optional)");
+DEFINE_uint64(num_rows, 10000, "How many rows to write and read");
+DEFINE_uint64(batch_num_rows, 10000, "How many rows batch for multi-gets and multi-puts");
+DEFINE_uint64(report_num_rows, 10000, "How frequent we should report the progress");
+DEFINE_bool(puts, true, "Whether to perform puts");
+DEFINE_bool(gets, true, "Whether to perform gets");
+DEFINE_bool(multigets, true, "Whether to perform multi-gets");
+DEFINE_bool(scans, true, "Whether to perform scans");
+DEFINE_bool(display_results, false, "Whether to display the Results from Gets");
+DEFINE_int32(threads, 6, "How many cpu threads");
+
+std::unique_ptr<Put> MakePut(const std::string &row) {
+  auto put = std::make_unique<Put>(row);
+  put->AddColumn("f", "q", row);
+  return std::move(put);
+}
+
+std::string Row(const std::string &prefix, uint64_t i) {
+  auto suf = folly::to<std::string>(i);
+  return prefix + suf;
+}
+
+void ValidateResult(const Result &result, const std::string &row) {
+  CHECK(!result.IsEmpty());
+  CHECK_EQ(result.Row(), row);
+  CHECK_EQ(result.Size(), 1);
+  CHECK_EQ(result.Value("f", "q").value(), row);
+}
+
+int main(int argc, char *argv[]) {
+  gflags::SetUsageMessage("Simple client to get a single row from HBase on the comamnd line");
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  google::InitGoogleLogging(argv[0]);
+  google::InstallFailureSignalHandler();
+  FLAGS_logtostderr = 1;
+  FLAGS_stderrthreshold = 1;
+
+  std::shared_ptr<Configuration> conf = nullptr;
+  if (FLAGS_conf == "") {
+    // Configuration
+    conf = std::make_shared<Configuration>();
+    conf->Set("hbase.zookeeper.quorum", FLAGS_zookeeper);
+    conf->SetInt("hbase.client.cpu.thread.pool.size", FLAGS_threads);
+  } else {
+    setenv("HBASE_CONF", FLAGS_conf.c_str(), 1);
+    hbase::HBaseConfigurationLoader loader;
+    conf = std::make_shared<Configuration>(loader.LoadDefaultResources().value());
+  }
+
+  auto row = FLAGS_row;
+
+  auto tn = std::make_shared<TableName>(folly::to<TableName>(FLAGS_table));
+  auto num_puts = FLAGS_num_rows;
+
+  auto client = std::make_unique<Client>(*conf);
+  auto table = client->Table(*tn);
+
+  auto start_ns = TimeUtil::GetNowNanos();
+
+  // Do the Put requests
+  if (FLAGS_puts) {
+    LOG(INFO) << "Sending put requests";
+    for (uint64_t i = 0; i < num_puts; i++) {
+      table->Put(*MakePut(Row(FLAGS_row, i)));
+      if (i != 0 && i % FLAGS_report_num_rows == 0) {
+        LOG(INFO) << "Sent  " << i << " Put requests in " << TimeUtil::ElapsedMillis(start_ns)
+                  << " ms.";
+      }
+    }
+
+    LOG(INFO) << "Successfully sent  " << num_puts << " Put requests in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+
+  // Do the Get requests
+  if (FLAGS_gets) {
+    LOG(INFO) << "Sending get requests";
+    start_ns = TimeUtil::GetNowNanos();
+    for (uint64_t i = 0; i < num_puts; i++) {
+      auto row = Row(FLAGS_row, i);
+      auto result = table->Get(Get{row});
+      if (FLAGS_display_results) {
+        LOG(INFO) << result->DebugString();
+      } else if (i != 0 && i % FLAGS_report_num_rows == 0) {
+        LOG(INFO) << "Sent  " << i << " Get requests in " << TimeUtil::ElapsedMillis(start_ns)
+                  << " ms.";
+      }
+      ValidateResult(*result, row);
+    }
+
+    LOG(INFO) << "Successfully sent  " << num_puts << " Get requests in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+
+  // Do the Multi-Gets
+  if (FLAGS_multigets) {
+    LOG(INFO) << "Sending multi-get requests";
+    start_ns = TimeUtil::GetNowNanos();
+    std::vector<hbase::Get> gets;
+
+    for (uint64_t i = 0; i < num_puts;) {
+      gets.clear();
+      // accumulate batch_num_rows at a time
+      for (uint64_t j = 0; j < FLAGS_batch_num_rows && i < num_puts; ++j) {
+        hbase::Get get(Row(FLAGS_row, i));
+        gets.push_back(get);
+        i++;
+      }
+      auto results = table->Get(gets);
+
+      if (FLAGS_display_results) {
+        for (const auto &result : results) LOG(INFO) << result->DebugString();
+      } else if (i != 0 && i % FLAGS_report_num_rows == 0) {
+        LOG(INFO) << "Sent  " << i << " Multi-Get requests in " << TimeUtil::ElapsedMillis(start_ns)
+                  << " ms.";
+      }
+    }
+
+    LOG(INFO) << "Successfully sent  " << num_puts << " Multi-Get requests in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+  }
+
+  // Do the Scan
+  if (FLAGS_scans) {
+    LOG(INFO) << "Starting scanner";
+    start_ns = TimeUtil::GetNowNanos();
+    Scan scan{};
+    auto scanner = table->Scan(scan);
+
+    uint64_t i = 0;
+    auto r = scanner->Next();
+    while (r != nullptr) {
+      if (FLAGS_display_results) {
+        LOG(INFO) << r->DebugString();
+      }
+      r = scanner->Next();
+      i++;
+      if (!FLAGS_display_results && i != 0 && i % FLAGS_report_num_rows == 0) {
+        LOG(INFO) << "Scan iterated over " << i << " results " << TimeUtil::ElapsedMillis(start_ns)
+                  << " ms.";
+      }
+    }
+
+    LOG(INFO) << "Successfully iterated over  " << i << " Scan results in "
+              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+    scanner->Close();
+  }
+
+  table->Close();
+  client->Close();
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/table.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/table.cc b/hbase-native-client/src/hbase/client/table.cc
new file mode 100644
index 0000000..e877ec8
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/table.cc
@@ -0,0 +1,144 @@
+/*
+ * 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 "hbase/client/table.h"
+
+#include <chrono>
+#include <limits>
+#include <utility>
+#include <vector>
+
+#include "hbase/client/async-connection.h"
+#include "hbase/client/async-table-result-scanner.h"
+#include "hbase/client/request-converter.h"
+#include "hbase/client/response-converter.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/security/user.h"
+#include "hbase/serde/server-name.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::pb::TableName;
+using hbase::security::User;
+using std::chrono::milliseconds;
+
+namespace hbase {
+
+Table::Table(const TableName &table_name, std::shared_ptr<AsyncConnection> async_connection)
+    : table_name_(std::make_shared<TableName>(table_name)),
+      async_connection_(async_connection),
+      conf_(async_connection->conf()) {
+  async_table_ = std::make_unique<RawAsyncTable>(table_name_, async_connection);
+}
+
+Table::~Table() {}
+
+std::shared_ptr<hbase::Result> Table::Get(const hbase::Get &get) {
+  auto context = async_table_->Get(get);
+  return context.get(operation_timeout());
+}
+
+std::shared_ptr<ResultScanner> Table::Scan(const hbase::Scan &scan) {
+  auto max_cache_size = ResultSize2CacheSize(
+      scan.MaxResultSize() > 0 ? scan.MaxResultSize()
+                               : async_connection_->connection_conf()->scanner_max_result_size());
+  auto scanner = std::make_shared<AsyncTableResultScanner>(max_cache_size);
+  async_table_->Scan(scan, scanner);
+  return scanner;
+}
+
+int64_t Table::ResultSize2CacheSize(int64_t max_results_size) const {
+  // * 2 if possible
+  return max_results_size > (std::numeric_limits<int64_t>::max() / 2) ? max_results_size
+                                                                      : max_results_size * 2;
+}
+
+void Table::Put(const hbase::Put &put) {
+  auto future = async_table_->Put(put);
+  future.get(operation_timeout());
+}
+
+bool Table::CheckAndPut(const std::string &row, const std::string &family,
+                        const std::string &qualifier, const std::string &value,
+                        const hbase::Put &put, const pb::CompareType &compare_op) {
+  auto context = async_table_->CheckAndPut(row, family, qualifier, value, put, compare_op);
+  return context.get(operation_timeout());
+}
+
+bool Table::CheckAndDelete(const std::string &row, const std::string &family,
+                           const std::string &qualifier, const std::string &value,
+                           const hbase::Delete &del, const pb::CompareType &compare_op) {
+  auto context = async_table_->CheckAndDelete(row, family, qualifier, value, del, compare_op);
+  return context.get(operation_timeout());
+}
+
+void Table::Delete(const hbase::Delete &del) {
+  auto future = async_table_->Delete(del);
+  future.get(operation_timeout());
+}
+
+std::shared_ptr<hbase::Result> Table::Increment(const hbase::Increment &increment) {
+  auto context = async_table_->Increment(increment);
+  return context.get(operation_timeout());
+}
+
+std::shared_ptr<hbase::Result> Table::Append(const hbase::Append &append) {
+  auto context = async_table_->Append(append);
+  return context.get(operation_timeout());
+}
+
+milliseconds Table::operation_timeout() const {
+  return TimeUtil::ToMillis(async_connection_->connection_conf()->operation_timeout());
+}
+
+void Table::Close() { async_table_->Close(); }
+
+std::shared_ptr<RegionLocation> Table::GetRegionLocation(const std::string &row) {
+  return async_connection_->region_locator()->LocateRegion(*table_name_, row).get();
+}
+
+std::vector<std::shared_ptr<hbase::Result>> Table::Get(const std::vector<hbase::Get> &gets) {
+  auto tresults = async_table_->Get(gets).get(operation_timeout());
+  std::vector<std::shared_ptr<hbase::Result>> results{};
+  uint32_t num = 0;
+  for (auto tresult : tresults) {
+    if (tresult.hasValue()) {
+      results.push_back(tresult.value());
+    } else if (tresult.hasException()) {
+      LOG(ERROR) << "Caught exception:- " << tresult.exception().what() << " for "
+                 << gets[num++].row();
+      throw tresult.exception();
+    }
+  }
+  return results;
+}
+
+void Table::Put(const std::vector<hbase::Put> &puts) {
+  auto tresults = async_table_->Put(puts).get(operation_timeout());
+  uint32_t num = 0;
+  for (auto tresult : tresults) {
+    if (tresult.hasException()) {
+      LOG(ERROR) << "Caught exception:- " << tresult.exception().what() << " for "
+                 << puts[num++].row();
+      throw tresult.exception();
+    }
+  }
+  return;
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/time-range-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/time-range-test.cc b/hbase-native-client/src/hbase/client/time-range-test.cc
new file mode 100644
index 0000000..a3db877
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/time-range-test.cc
@@ -0,0 +1,48 @@
+/*
+ * 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 "hbase/client/time-range.h"
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+using namespace hbase;
+
+TEST(TimeRange, DefaultObject) {
+  TimeRange *timerange_def = nullptr;
+  ASSERT_NO_THROW(timerange_def = new TimeRange());
+
+  EXPECT_EQ(0, timerange_def->MinTimeStamp());
+  EXPECT_EQ(std::numeric_limits<int64_t>::max(), timerange_def->MaxTimeStamp());
+  EXPECT_NE(1000, timerange_def->MinTimeStamp());
+  EXPECT_NE(2000, timerange_def->MaxTimeStamp());
+  delete timerange_def;
+  timerange_def = nullptr;
+}
+
+TEST(TimeRange, Exception) {
+  // Negative Min TS
+  ASSERT_THROW(TimeRange(-1000, 2000), std::runtime_error);
+
+  // Negative Max TS
+  ASSERT_THROW(TimeRange(1000, -2000), std::runtime_error);
+
+  // Min TS > Max TS
+  ASSERT_THROW(TimeRange(10000, 2000), std::runtime_error);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/time-range.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/time-range.cc b/hbase-native-client/src/hbase/client/time-range.cc
new file mode 100644
index 0000000..b53e6f9
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/time-range.cc
@@ -0,0 +1,73 @@
+/*
+ * 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 "hbase/client/time-range.h"
+#include <limits>
+#include <stdexcept>
+#include <string>
+
+namespace hbase {
+
+TimeRange::TimeRange()
+    : min_timestamp_(0L), max_timestamp_(std::numeric_limits<int64_t>::max()), all_time_(true) {}
+
+TimeRange::TimeRange(const TimeRange &tr) {
+  this->all_time_ = tr.all_time_;
+  this->max_timestamp_ = tr.max_timestamp_;
+  this->min_timestamp_ = tr.min_timestamp_;
+}
+
+TimeRange &TimeRange::operator=(const TimeRange &tr) {
+  this->all_time_ = tr.all_time_;
+  this->max_timestamp_ = tr.max_timestamp_;
+  this->min_timestamp_ = tr.min_timestamp_;
+  return *this;
+}
+
+TimeRange::~TimeRange() {}
+
+TimeRange::TimeRange(int64_t min_timestamp) {
+  this->min_timestamp_ = min_timestamp;
+  this->max_timestamp_ = std::numeric_limits<int64_t>::max();
+  this->all_time_ = false;
+}
+
+TimeRange::TimeRange(int64_t min_timestamp, int64_t max_timestamp) {
+  if (min_timestamp < 0 || max_timestamp < 0) {
+    throw std::runtime_error("Timestamp cannot be negative. min_timestamp: " +
+                             std::to_string(min_timestamp) + ", max_timestamp:" +
+                             std::to_string(max_timestamp));
+  }
+  if (max_timestamp < min_timestamp) {
+    throw std::runtime_error("max_timestamp [" + std::to_string(max_timestamp) +
+                             "] should be greater than min_timestamp [" +
+                             std::to_string(min_timestamp) + "]");
+  }
+
+  this->min_timestamp_ = min_timestamp;
+  this->max_timestamp_ = max_timestamp;
+  this->all_time_ = false;
+}
+
+int64_t TimeRange::MinTimeStamp() const { return this->min_timestamp_; }
+
+int64_t TimeRange::MaxTimeStamp() const { return this->max_timestamp_; }
+
+bool TimeRange::IsAllTime() const { return this->all_time_; }
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/zk-util-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/zk-util-test.cc b/hbase-native-client/src/hbase/client/zk-util-test.cc
new file mode 100644
index 0000000..83d22fb
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/zk-util-test.cc
@@ -0,0 +1,50 @@
+/*
+ * 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 <gtest/gtest.h>
+
+#include "hbase/client/zk-util.h"
+
+using hbase::Configuration;
+using hbase::ZKUtil;
+
+TEST(ZKUtilTest, ParseZooKeeperQuorum) {
+  Configuration conf{};
+  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1");
+  conf.SetInt(ZKUtil::kHBaseZookeeperClientPort_, 100);
+
+  ASSERT_EQ("s1:100", ZKUtil::ParseZooKeeperQuorum(conf));
+
+  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1:42");
+
+  ASSERT_EQ("s1:42", ZKUtil::ParseZooKeeperQuorum(conf));
+
+  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1,s2,s3");
+  ASSERT_EQ("s1:100,s2:100,s3:100", ZKUtil::ParseZooKeeperQuorum(conf));
+
+  conf.Set(ZKUtil::kHBaseZookeeperQuorum_, "s1:42,s2:42,s3:42");
+  ASSERT_EQ("s1:42,s2:42,s3:42", ZKUtil::ParseZooKeeperQuorum(conf));
+}
+
+TEST(ZKUtilTest, MetaZNode) {
+  Configuration conf{};
+  ASSERT_EQ("/hbase/meta-region-server", ZKUtil::MetaZNode(conf));
+
+  conf.Set(ZKUtil::kHBaseZnodeParent_, "/hbase-secure");
+  ASSERT_EQ("/hbase-secure/meta-region-server", ZKUtil::MetaZNode(conf));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/zk-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/zk-util.cc b/hbase-native-client/src/hbase/client/zk-util.cc
new file mode 100644
index 0000000..38a0a2e
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/zk-util.cc
@@ -0,0 +1,62 @@
+/*
+ * 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 "hbase/client/zk-util.h"
+
+#include <folly/Conv.h>
+#include <boost/algorithm/string.hpp>
+
+#include <vector>
+
+namespace hbase {
+
+/**
+ * Returns a "proper" zookeeper quorum string, from hbase's broken quorum string formats. In
+ * hbase.zookeeper.quorum, the ports are not listed explicitly per server (eg. s1,s2,s3),
+ * however ZooKeeper expects the string of the format s1:2181,s2:2181,s3:2181. This code
+ * appends the "clientPort" to each node in the quorum string if not there.
+ */
+std::string ZKUtil::ParseZooKeeperQuorum(const hbase::Configuration& conf) {
+  auto zk_quorum = conf.Get(kHBaseZookeeperQuorum_, kDefHBaseZookeeperQuorum_);
+  auto zk_port = conf.GetInt(kHBaseZookeeperClientPort_, kDefHBaseZookeeperClientPort_);
+
+  std::vector<std::string> zk_quorum_parts;
+  boost::split(zk_quorum_parts, zk_quorum, boost::is_any_of(","), boost::token_compress_on);
+  std::vector<std::string> servers;
+  for (auto server : zk_quorum_parts) {
+    if (boost::contains(server, ":")) {
+      servers.push_back(server);
+    } else {
+      servers.push_back(server + ":" + folly::to<std::string>(zk_port));
+    }
+  }
+  return boost::join(servers, ",");
+}
+
+std::string ZKUtil::MetaZNode(const hbase::Configuration& conf) {
+  std::string zk_node = conf.Get(kHBaseZnodeParent_, kDefHBaseZnodeParent_) + "/";
+  zk_node += kHBaseMetaRegionServer_;
+  return zk_node;
+}
+
+int32_t ZKUtil::SessionTimeout(const hbase::Configuration& conf) {
+  return conf.GetInt(kHBaseZookeeperSessionTimeout_, kDefHBaseZookeeperSessionTimeout_);
+}
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/BUCK b/hbase-native-client/src/hbase/connection/BUCK
new file mode 100644
index 0000000..1a856fb
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/BUCK
@@ -0,0 +1,68 @@
+##
+# 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.
+
+# This is the library dealing with a single connection
+# to a single server.
+cxx_library(
+    name="connection",
+    srcs=[
+        "client-dispatcher.cc",
+        "client-handler.cc",
+        "connection-factory.cc",
+        "connection-pool.cc",
+        "pipeline.cc",
+        "request.cc",
+        "rpc-client.cc",
+        "sasl-handler.cc",
+        "sasl-util.cc",
+        "rpc-test-server.cc",
+        "rpc-test-server-handler.cc",
+        "rpc-fault-injector.cc",
+    ],
+    deps=[
+        "//include/hbase/connection:connection",
+        "//src/hbase/if:if",
+        "//src/hbase/utils:utils",
+        "//src/hbase/serde:serde",
+        "//src/hbase/security:security",
+        "//third-party:folly",
+        "//third-party:wangle",
+        "//src/hbase/exceptions:exceptions",
+    ],
+    compiler_flags=['-Weffc++'],
+    linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
+    exported_linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
+    visibility=[
+        '//src/hbase/client/...',
+    ],)
+cxx_test(
+    name="connection-pool-test",
+    srcs=[
+        "connection-pool-test.cc",
+    ],
+    deps=[
+        ":connection",
+    ],)
+cxx_test(
+    name="rpc-test",
+    srcs=[
+        "rpc-test.cc",
+    ],
+    deps=[
+        ":connection",
+    ],
+    run_test_separately=True,)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/client-dispatcher.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/client-dispatcher.cc b/hbase-native-client/src/hbase/connection/client-dispatcher.cc
new file mode 100644
index 0000000..302ab6b
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/client-dispatcher.cc
@@ -0,0 +1,115 @@
+/*
+ * 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 "hbase/connection/client-dispatcher.h"
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/Format.h>
+#include <folly/io/async/AsyncSocketException.h>
+#include <utility>
+
+#include "hbase/connection/rpc-connection.h"
+#include "hbase/exceptions/exception.h"
+
+using std::unique_ptr;
+
+namespace hbase {
+
+ClientDispatcher::ClientDispatcher(const std::string &server)
+    : current_call_id_(9), requests_(5000), server_(server), is_closed_(false) {}
+
+void ClientDispatcher::read(Context *ctx, unique_ptr<Response> in) {
+  VLOG(5) << "ClientDispatcher::read()";
+  auto call_id = in->call_id();
+  auto p = requests_.find_and_erase(call_id);
+
+  VLOG(3) << folly::sformat("Read hbase::Response, call_id: {}, hasException: {}, what: {}",
+                            in->call_id(), bool(in->exception()), in->exception().what());
+
+  if (in->exception()) {
+    p.setException(in->exception());
+  } else {
+    p.setValue(std::move(in));
+  }
+}
+
+void ClientDispatcher::readException(Context *ctx, folly::exception_wrapper e) {
+  VLOG(5) << "ClientDispatcher::readException()";
+  CloseAndCleanUpCalls();
+}
+
+void ClientDispatcher::readEOF(Context *ctx) {
+  VLOG(5) << "ClientDispatcher::readEOF()";
+  CloseAndCleanUpCalls();
+}
+
+folly::Future<unique_ptr<Response>> ClientDispatcher::operator()(unique_ptr<Request> arg) {
+  VLOG(5) << "ClientDispatcher::operator()";
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (is_closed_) {
+    throw ConnectionException("Connection closed already");
+  }
+
+  auto call_id = current_call_id_++;
+  arg->set_call_id(call_id);
+
+  // TODO: if the map is full (or we have more than hbase.client.perserver.requests.threshold)
+  // then throw ServerTooBusyException so that upper layers will retry.
+  auto &p = requests_[call_id];
+
+  auto f = p.getFuture();
+  p.setInterruptHandler([call_id, this](const folly::exception_wrapper &e) {
+    LOG(ERROR) << "e = " << call_id;
+    this->requests_.erase(call_id);
+    // TODO: call Promise::SetException()?
+  });
+
+  try {
+    this->pipeline_->write(std::move(arg));
+  } catch (const folly::AsyncSocketException &e) {
+    p.setException(folly::exception_wrapper{ConnectionException{folly::exception_wrapper{e}}});
+    /* clear folly::Promise to avoid overflow. */
+    requests_.erase(call_id);
+  }
+
+  return f;
+}
+
+void ClientDispatcher::CloseAndCleanUpCalls() {
+  VLOG(5) << "ClientDispatcher::CloseAndCleanUpCalls()";
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (is_closed_) {
+    return;
+  }
+  for (auto &pair : requests_) {
+    pair.second.setException(IOException{"Connection closed to server:" + server_});
+  }
+  requests_.clear();
+  is_closed_ = true;
+}
+
+folly::Future<folly::Unit> ClientDispatcher::close() {
+  CloseAndCleanUpCalls();
+  return ClientDispatcherBase::close();
+}
+
+folly::Future<folly::Unit> ClientDispatcher::close(Context *ctx) {
+  CloseAndCleanUpCalls();
+  return ClientDispatcherBase::close(ctx);
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/client-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/client-handler.cc b/hbase-native-client/src/hbase/connection/client-handler.cc
new file mode 100644
index 0000000..c963c20
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/client-handler.cc
@@ -0,0 +1,145 @@
+/*
+ * 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 "hbase/connection/client-handler.h"
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/Likely.h>
+#include <folly/io/async/AsyncSocketException.h>
+#include <glog/logging.h>
+#include <string>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/RPC.pb.h"
+
+using google::protobuf::Message;
+
+namespace hbase {
+
+ClientHandler::ClientHandler(std::string user_name, std::shared_ptr<Codec> codec,
+                             std::shared_ptr<Configuration> conf, const std::string &server)
+    : user_name_(user_name),
+      serde_(codec),
+      conf_(conf),
+      server_(server),
+      once_flag_(std::make_unique<std::once_flag>()),
+      resp_msgs_(
+          std::make_unique<concurrent_map<uint32_t, std::shared_ptr<google::protobuf::Message>>>(
+              5000)) {}
+
+void ClientHandler::read(Context *ctx, std::unique_ptr<folly::IOBuf> buf) {
+  if (LIKELY(buf != nullptr)) {
+    buf->coalesce();
+    auto received = std::make_unique<Response>();
+    pb::ResponseHeader header;
+
+    int used_bytes = serde_.ParseDelimited(buf.get(), &header);
+    VLOG(3) << "Read RPC ResponseHeader size=" << used_bytes << " call_id=" << header.call_id()
+            << " has_exception=" << header.has_exception() << ", server: " << server_;
+
+    auto resp_msg = resp_msgs_->find_and_erase(header.call_id());
+
+    // set the call_id.
+    // This will be used to by the dispatcher to match up
+    // the promise with the response.
+    received->set_call_id(header.call_id());
+
+    // If there was an exception then there's no
+    // data left on the wire.
+    if (header.has_exception() == false) {
+      buf->trimStart(used_bytes);
+
+      int cell_block_length = 0;
+      used_bytes = serde_.ParseDelimited(buf.get(), resp_msg.get());
+      if (header.has_cell_block_meta() && header.cell_block_meta().has_length()) {
+        cell_block_length = header.cell_block_meta().length();
+      }
+
+      VLOG(3) << "Read RPCResponse, buf length:" << buf->length()
+              << ", header PB length:" << used_bytes << ", cell_block length:" << cell_block_length
+              << ", server: " << server_;
+
+      // Make sure that bytes were parsed.
+      CHECK((used_bytes + cell_block_length) == buf->length());
+
+      if (cell_block_length > 0) {
+        auto cell_scanner = serde_.CreateCellScanner(std::move(buf), used_bytes, cell_block_length);
+        received->set_cell_scanner(std::shared_ptr<CellScanner>{cell_scanner.release()});
+      }
+
+      received->set_resp_msg(resp_msg);
+    } else {
+      hbase::pb::ExceptionResponse exceptionResponse = header.exception();
+
+      std::string what;
+      std::string exception_class_name = exceptionResponse.has_exception_class_name()
+                                             ? exceptionResponse.exception_class_name()
+                                             : "";
+      std::string stack_trace =
+          exceptionResponse.has_stack_trace() ? exceptionResponse.stack_trace() : "";
+      what.append(stack_trace);
+
+      auto remote_exception = std::make_unique<RemoteException>(what);
+      remote_exception->set_exception_class_name(exception_class_name)
+          ->set_stack_trace(stack_trace)
+          ->set_hostname(exceptionResponse.has_hostname() ? exceptionResponse.hostname() : "")
+          ->set_port(exceptionResponse.has_port() ? exceptionResponse.port() : 0);
+      if (exceptionResponse.has_do_not_retry()) {
+        remote_exception->set_do_not_retry(exceptionResponse.do_not_retry());
+      }
+
+      VLOG(3) << "Exception RPC ResponseHeader, call_id=" << header.call_id()
+              << " exception.what=" << remote_exception->what()
+              << ", do_not_retry=" << remote_exception->do_not_retry() << ", server: " << server_;
+      received->set_exception(folly::exception_wrapper{*remote_exception});
+    }
+    ctx->fireRead(std::move(received));
+  }
+}
+
+folly::Future<folly::Unit> ClientHandler::write(Context *ctx, std::unique_ptr<Request> r) {
+  /* for RPC test, there's no need to send connection header */
+  if (!conf_->GetBool(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE,
+                      RpcSerde::DEFAULT_HBASE_CLIENT_RPC_TEST_MODE)) {
+    // We need to send the header once.
+    // So use call_once to make sure that only one thread wins this.
+    std::call_once((*once_flag_), [ctx, this]() {
+      VLOG(3) << "Writing RPC Header to server: " << server_;
+      auto header = serde_.Header(user_name_);
+      ctx->fireWrite(std::move(header));
+    });
+  }
+
+  VLOG(3) << "Writing RPC Request:" << r->DebugString() << ", server: " << server_;
+
+  // Now store the call id to response.
+  resp_msgs_->insert(std::make_pair(r->call_id(), r->resp_msg()));
+
+  try {
+    // Send the data down the pipeline.
+    return ctx->fireWrite(serde_.Request(r->call_id(), r->method(), r->req_msg().get()));
+  } catch (const folly::AsyncSocketException &e) {
+    /* clear protobuf::Message to avoid overflow. */
+    resp_msgs_->erase(r->call_id());
+    throw e;
+  }
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/connection-factory.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/connection-factory.cc b/hbase-native-client/src/hbase/connection/connection-factory.cc
new file mode 100644
index 0000000..14ac22b
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/connection-factory.cc
@@ -0,0 +1,88 @@
+/*
+ * 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 <folly/Conv.h>
+#include <glog/logging.h>
+#include <wangle/channel/Handler.h>
+
+#include <chrono>
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/SocketAddress.h>
+#include <folly/io/async/AsyncSocketException.h>
+
+#include "hbase/connection/client-dispatcher.h"
+#include "hbase/connection/connection-factory.h"
+#include "hbase/connection/pipeline.h"
+#include "hbase/connection/sasl-handler.h"
+#include "hbase/connection/service.h"
+#include "hbase/exceptions/exception.h"
+
+using std::chrono::milliseconds;
+using std::chrono::nanoseconds;
+
+namespace hbase {
+
+ConnectionFactory::ConnectionFactory(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                                     std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                                     std::shared_ptr<Codec> codec,
+                                     std::shared_ptr<Configuration> conf,
+                                     nanoseconds connect_timeout)
+    : connect_timeout_(connect_timeout),
+      io_executor_(io_executor),
+      cpu_executor_(cpu_executor),
+      conf_(conf),
+      pipeline_factory_(std::make_shared<RpcPipelineFactory>(codec, conf)) {}
+
+std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> ConnectionFactory::MakeBootstrap() {
+  auto client = std::make_shared<wangle::ClientBootstrap<SerializePipeline>>();
+  client->group(io_executor_);
+  client->pipelineFactory(pipeline_factory_);
+
+  // TODO: Opened https://github.com/facebook/wangle/issues/85 in wangle so that we can set socket
+  //  options like TCP_NODELAY, SO_KEEPALIVE, CONNECT_TIMEOUT_MILLIS, etc.
+
+  return client;
+}
+
+std::shared_ptr<HBaseService> ConnectionFactory::Connect(
+    std::shared_ptr<RpcConnection> rpc_connection,
+    std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap,
+    const std::string &hostname, uint16_t port) {
+  // connection should happen from an IO thread
+  try {
+    auto future = via(io_executor_.get()).then([=]() {
+      VLOG(1) << "Connecting to server: " << hostname << ":" << port;
+      return client_bootstrap->connect(folly::SocketAddress(hostname, port, true),
+                                       std::chrono::duration_cast<milliseconds>(connect_timeout_));
+    });
+
+    // See about using shared promise for this.
+    auto pipeline = future.get();
+
+    VLOG(1) << "Connected to server: " << hostname << ":" << port;
+    auto dispatcher =
+        std::make_shared<ClientDispatcher>(hostname + ":" + folly::to<std::string>(port));
+    dispatcher->setPipeline(pipeline);
+    return dispatcher;
+  } catch (const folly::AsyncSocketException &e) {
+    throw ConnectionException(folly::exception_wrapper{e});
+  }
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/connection-pool-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/connection-pool-test.cc b/hbase-native-client/src/hbase/connection/connection-pool-test.cc
new file mode 100644
index 0000000..5886a42
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/connection-pool-test.cc
@@ -0,0 +1,140 @@
+/*
+ * 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 <folly/Logging.h>
+#include <gmock/gmock.h>
+
+#include "hbase/connection/connection-factory.h"
+#include "hbase/connection/connection-id.h"
+#include "hbase/connection/connection-pool.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/server-name.h"
+
+using hbase::pb::ServerName;
+using ::testing::Return;
+using ::testing::_;
+using hbase::ConnectionFactory;
+using hbase::ConnectionPool;
+using hbase::ConnectionId;
+using hbase::HBaseService;
+using hbase::Request;
+using hbase::Response;
+using hbase::RpcConnection;
+using hbase::SerializePipeline;
+
+class MockConnectionFactory : public ConnectionFactory {
+ public:
+  MockConnectionFactory() : ConnectionFactory(nullptr, nullptr, nullptr, nullptr) {}
+  MOCK_METHOD0(MakeBootstrap, std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>>());
+  MOCK_METHOD4(Connect, std::shared_ptr<HBaseService>(
+                            std::shared_ptr<RpcConnection> rpc_connection,
+                            std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>>,
+                            const std::string &hostname, uint16_t port));
+};
+
+class MockBootstrap : public wangle::ClientBootstrap<SerializePipeline> {};
+
+class MockService : public HBaseService {
+ public:
+  folly::Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> req) override {
+    return folly::makeFuture<std::unique_ptr<Response>>(
+        std::make_unique<Response>(do_operation(req.get())));
+  }
+  MOCK_METHOD1(do_operation, Response(Request *));
+};
+
+TEST(TestConnectionPool, TestOnlyCreateOnce) {
+  auto hostname = std::string{"hostname"};
+  auto mock_boot = std::make_shared<MockBootstrap>();
+  auto mock_service = std::make_shared<MockService>();
+  auto mock_cf = std::make_shared<MockConnectionFactory>();
+  uint32_t port{999};
+
+  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(1).WillRepeatedly(Return(mock_service));
+  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(1).WillRepeatedly(Return(mock_boot));
+  EXPECT_CALL((*mock_service), do_operation(_)).Times(1).WillRepeatedly(Return(Response{}));
+  ConnectionPool cp{mock_cf};
+
+  auto remote_id = std::make_shared<ConnectionId>(hostname, port);
+  auto result = cp.GetConnection(remote_id);
+  ASSERT_TRUE(result != nullptr);
+  result = cp.GetConnection(remote_id);
+  result->SendRequest(nullptr);
+}
+
+TEST(TestConnectionPool, TestOnlyCreateMultipleDispose) {
+  std::string hostname_one{"hostname"};
+  std::string hostname_two{"hostname_two"};
+  uint32_t port{999};
+
+  auto mock_boot = std::make_shared<MockBootstrap>();
+  auto mock_service = std::make_shared<MockService>();
+  auto mock_cf = std::make_shared<MockConnectionFactory>();
+
+  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(2).WillRepeatedly(Return(mock_service));
+  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(2).WillRepeatedly(Return(mock_boot));
+  EXPECT_CALL((*mock_service), do_operation(_)).Times(4).WillRepeatedly(Return(Response{}));
+  ConnectionPool cp{mock_cf};
+
+  {
+    auto remote_id = std::make_shared<ConnectionId>(hostname_one, port);
+    auto result_one = cp.GetConnection(remote_id);
+    result_one->SendRequest(nullptr);
+    auto remote_id2 = std::make_shared<ConnectionId>(hostname_two, port);
+    auto result_two = cp.GetConnection(remote_id2);
+    result_two->SendRequest(nullptr);
+  }
+  auto remote_id = std::make_shared<ConnectionId>(hostname_one, port);
+  auto result_one = cp.GetConnection(remote_id);
+  result_one->SendRequest(nullptr);
+  auto remote_id2 = std::make_shared<ConnectionId>(hostname_two, port);
+  auto result_two = cp.GetConnection(remote_id2);
+  result_two->SendRequest(nullptr);
+}
+
+TEST(TestConnectionPool, TestCreateOneConnectionForOneService) {
+  std::string hostname{"hostname"};
+  uint32_t port{999};
+  std::string service1{"service1"};
+  std::string service2{"service2"};
+
+  auto mock_boot = std::make_shared<MockBootstrap>();
+  auto mock_service = std::make_shared<MockService>();
+  auto mock_cf = std::make_shared<MockConnectionFactory>();
+
+  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(2).WillRepeatedly(Return(mock_service));
+  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(2).WillRepeatedly(Return(mock_boot));
+  EXPECT_CALL((*mock_service), do_operation(_)).Times(4).WillRepeatedly(Return(Response{}));
+  ConnectionPool cp{mock_cf};
+
+  {
+    auto remote_id = std::make_shared<ConnectionId>(hostname, port, service1);
+    auto result_one = cp.GetConnection(remote_id);
+    result_one->SendRequest(nullptr);
+    auto remote_id2 = std::make_shared<ConnectionId>(hostname, port, service2);
+    auto result_two = cp.GetConnection(remote_id2);
+    result_two->SendRequest(nullptr);
+  }
+  auto remote_id = std::make_shared<ConnectionId>(hostname, port, service1);
+  auto result_one = cp.GetConnection(remote_id);
+  result_one->SendRequest(nullptr);
+  auto remote_id2 = std::make_shared<ConnectionId>(hostname, port, service2);
+  auto result_two = cp.GetConnection(remote_id2);
+  result_two->SendRequest(nullptr);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/connection-pool.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/connection-pool.cc b/hbase-native-client/src/hbase/connection/connection-pool.cc
new file mode 100644
index 0000000..92e87f8
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/connection-pool.cc
@@ -0,0 +1,119 @@
+/*
+ * 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 "hbase/connection/connection-pool.h"
+
+#include <folly/Conv.h>
+#include <folly/Logging.h>
+#include <wangle/service/Service.h>
+
+#include <memory>
+#include <string>
+#include <utility>
+
+using std::chrono::nanoseconds;
+
+namespace hbase {
+
+ConnectionPool::ConnectionPool(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                               std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                               std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
+                               nanoseconds connect_timeout)
+    : cf_(std::make_shared<ConnectionFactory>(io_executor, cpu_executor, codec, conf,
+                                              connect_timeout)),
+      connections_(),
+      map_mutex_(),
+      conf_(conf) {}
+ConnectionPool::ConnectionPool(std::shared_ptr<ConnectionFactory> cf)
+    : cf_(cf), connections_(), map_mutex_() {}
+
+ConnectionPool::~ConnectionPool() {}
+
+std::shared_ptr<RpcConnection> ConnectionPool::GetConnection(
+    std::shared_ptr<ConnectionId> remote_id) {
+  // Try and get th cached connection.
+  auto found_ptr = GetCachedConnection(remote_id);
+
+  // If there's no connection then create it.
+  if (found_ptr == nullptr) {
+    found_ptr = GetNewConnection(remote_id);
+  }
+  return found_ptr;
+}
+
+std::shared_ptr<RpcConnection> ConnectionPool::GetCachedConnection(
+    std::shared_ptr<ConnectionId> remote_id) {
+  folly::SharedMutexWritePriority::ReadHolder holder(map_mutex_);
+  auto found = connections_.find(remote_id);
+  if (found == connections_.end()) {
+    return nullptr;
+  }
+  return found->second;
+}
+
+std::shared_ptr<RpcConnection> ConnectionPool::GetNewConnection(
+    std::shared_ptr<ConnectionId> remote_id) {
+  // Grab the upgrade lock. While we are double checking other readers can
+  // continue on
+  folly::SharedMutexWritePriority::UpgradeHolder u_holder{map_mutex_};
+
+  // Now check if someone else created the connection before we got the lock
+  // This is safe since we hold the upgrade lock.
+  // upgrade lock is more power than the reader lock.
+  auto found = connections_.find(remote_id);
+  if (found != connections_.end() && found->second != nullptr) {
+    return found->second;
+  } else {
+    // Yeah it looks a lot like there's no connection
+    folly::SharedMutexWritePriority::WriteHolder w_holder{std::move(u_holder)};
+
+    // Make double sure there are not stale connections hanging around.
+    connections_.erase(remote_id);
+
+    /* create new connection */
+    auto connection = std::make_shared<RpcConnection>(remote_id, cf_);
+
+    connections_.insert(std::make_pair(remote_id, connection));
+
+    return connection;
+  }
+}
+
+void ConnectionPool::Close(std::shared_ptr<ConnectionId> remote_id) {
+  folly::SharedMutexWritePriority::WriteHolder holder{map_mutex_};
+  DLOG(INFO) << "Closing RPC Connection to host:" << remote_id->host()
+             << ", port:" << folly::to<std::string>(remote_id->port());
+
+  auto found = connections_.find(remote_id);
+  if (found == connections_.end() || found->second == nullptr) {
+    return;
+  }
+  found->second->Close();
+  connections_.erase(found);
+}
+
+void ConnectionPool::Close() {
+  folly::SharedMutexWritePriority::WriteHolder holder{map_mutex_};
+  for (auto &item : connections_) {
+    auto &con = item.second;
+    con->Close();
+  }
+  connections_.clear();
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/pipeline.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/pipeline.cc b/hbase-native-client/src/hbase/connection/pipeline.cc
new file mode 100644
index 0000000..45ac0c0
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/pipeline.cc
@@ -0,0 +1,55 @@
+/*
+ * 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 "hbase/connection/pipeline.h"
+
+#include <folly/Logging.h>
+#include <wangle/channel/AsyncSocketHandler.h>
+#include <wangle/channel/EventBaseHandler.h>
+#include <wangle/channel/OutputBufferingHandler.h>
+#include <wangle/codec/LengthFieldBasedFrameDecoder.h>
+
+#include "hbase/connection/client-handler.h"
+#include "hbase/connection/sasl-handler.h"
+
+namespace hbase {
+
+RpcPipelineFactory::RpcPipelineFactory(std::shared_ptr<Codec> codec,
+                                       std::shared_ptr<Configuration> conf)
+    : user_util_(), codec_(codec), conf_(conf) {}
+SerializePipeline::Ptr RpcPipelineFactory::newPipeline(
+    std::shared_ptr<folly::AsyncTransportWrapper> sock) {
+  folly::SocketAddress addr;  // for logging
+  sock->getPeerAddress(&addr);
+
+  auto pipeline = SerializePipeline::create();
+  pipeline->addBack(wangle::AsyncSocketHandler{sock});
+  pipeline->addBack(wangle::EventBaseHandler{});
+  bool secure = false;
+  /* for RPC test, there's no need to setup Sasl */
+  if (!conf_->GetBool(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE,
+                      RpcSerde::DEFAULT_HBASE_CLIENT_RPC_TEST_MODE)) {
+    secure = security::User::IsSecurityEnabled(*conf_);
+    pipeline->addBack(SaslHandler{user_util_.user_name(secure), conf_});
+  }
+  pipeline->addBack(wangle::LengthFieldBasedFrameDecoder{});
+  pipeline->addBack(ClientHandler{user_util_.user_name(secure), codec_, conf_, addr.describe()});
+  pipeline->finalize();
+  return pipeline;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/request.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/request.cc b/hbase-native-client/src/hbase/connection/request.cc
new file mode 100644
index 0000000..91a6119
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/request.cc
@@ -0,0 +1,46 @@
+/*
+ * 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 "hbase/connection/request.h"
+
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+Request::Request(std::shared_ptr<google::protobuf::Message> req,
+                 std::shared_ptr<google::protobuf::Message> resp, std::string method)
+    : req_msg_(req), resp_msg_(resp), method_(method), call_id_(0) {}
+
+std::unique_ptr<Request> Request::get() {
+  return std::make_unique<Request>(std::make_shared<hbase::pb::GetRequest>(),
+                                   std::make_shared<hbase::pb::GetResponse>(), "Get");
+}
+std::unique_ptr<Request> Request::mutate() {
+  return std::make_unique<Request>(std::make_shared<hbase::pb::MutateRequest>(),
+                                   std::make_shared<hbase::pb::MutateResponse>(), "Mutate");
+}
+std::unique_ptr<Request> Request::scan() {
+  return std::make_unique<Request>(std::make_shared<hbase::pb::ScanRequest>(),
+                                   std::make_shared<hbase::pb::ScanResponse>(), "Scan");
+}
+std::unique_ptr<Request> Request::multi() {
+  return std::make_unique<Request>(std::make_shared<hbase::pb::MultiRequest>(),
+                                   std::make_shared<hbase::pb::MultiResponse>(), "Multi");
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/rpc-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/rpc-client.cc b/hbase-native-client/src/hbase/connection/rpc-client.cc
new file mode 100644
index 0000000..d73829e
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/rpc-client.cc
@@ -0,0 +1,119 @@
+/*
+ * 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 "hbase/connection/rpc-client.h"
+
+#include <folly/Format.h>
+#include <folly/Logging.h>
+#include <folly/futures/Future.h>
+#include <unistd.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+#include "hbase/exceptions/exception.h"
+
+using hbase::security::User;
+using std::chrono::nanoseconds;
+
+namespace hbase {
+
+RpcClient::RpcClient(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                     std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                     std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
+                     nanoseconds connect_timeout)
+    : io_executor_(io_executor), conf_(conf) {
+  cp_ = std::make_shared<ConnectionPool>(io_executor_, cpu_executor, codec, conf, connect_timeout);
+}
+
+void RpcClient::Close() { io_executor_->stop(); }
+
+std::unique_ptr<Response> RpcClient::SyncCall(const std::string& host, uint16_t port,
+                                              std::unique_ptr<Request> req,
+                                              std::shared_ptr<User> ticket) {
+  return AsyncCall(host, port, std::move(req), ticket).get();
+}
+
+std::unique_ptr<Response> RpcClient::SyncCall(const std::string& host, uint16_t port,
+                                              std::unique_ptr<Request> req,
+                                              std::shared_ptr<User> ticket,
+                                              const std::string& service_name) {
+  return AsyncCall(host, port, std::move(req), ticket, service_name).get();
+}
+
+folly::Future<std::unique_ptr<Response>> RpcClient::AsyncCall(const std::string& host,
+                                                              uint16_t port,
+                                                              std::unique_ptr<Request> req,
+                                                              std::shared_ptr<User> ticket) {
+  auto remote_id = std::make_shared<ConnectionId>(host, port, ticket);
+  return SendRequest(remote_id, std::move(req));
+}
+
+folly::Future<std::unique_ptr<Response>> RpcClient::AsyncCall(const std::string& host,
+                                                              uint16_t port,
+                                                              std::unique_ptr<Request> req,
+                                                              std::shared_ptr<User> ticket,
+                                                              const std::string& service_name) {
+  auto remote_id = std::make_shared<ConnectionId>(host, port, ticket, service_name);
+  return SendRequest(remote_id, std::move(req));
+}
+
+/**
+ * There are two cases for ConnectionException:
+ * 1. The first time connection
+ * establishment, i.e. GetConnection(remote_id), AsyncSocketException being a cause.
+ * 2. Writing request down the pipeline, i.e. RpcConnection::SendRequest, AsyncSocketException being
+ * a cause as well.
+ */
+folly::Future<std::unique_ptr<Response>> RpcClient::SendRequest(
+    std::shared_ptr<ConnectionId> remote_id, std::unique_ptr<Request> req) {
+  try {
+    return GetConnection(remote_id)
+        ->SendRequest(std::move(req))
+        .onError([&, this](const folly::exception_wrapper& ew) {
+          VLOG(3) << folly::sformat("RpcClient Exception: {}", ew.what());
+          ew.with_exception([&, this](const hbase::ConnectionException& re) {
+            /* bad connection, remove it from pool. */
+            cp_->Close(remote_id);
+          });
+          return GetFutureWithException(ew);
+        });
+  } catch (const ConnectionException& e) {
+    CHECK(e.cause().get_exception() != nullptr);
+    VLOG(3) << folly::sformat("RpcClient Exception: {}", e.cause().what());
+    /* bad connection, remove it from pool. */
+    cp_->Close(remote_id);
+    return GetFutureWithException(e);
+  }
+}
+
+template <typename EXCEPTION>
+folly::Future<std::unique_ptr<Response>> RpcClient::GetFutureWithException(const EXCEPTION& e) {
+  return GetFutureWithException(folly::exception_wrapper{e});
+}
+
+folly::Future<std::unique_ptr<Response>> RpcClient::GetFutureWithException(
+    const folly::exception_wrapper& ew) {
+  folly::Promise<std::unique_ptr<Response>> promise;
+  auto future = promise.getFuture();
+  promise.setException(ew);
+  return future;
+}
+
+std::shared_ptr<RpcConnection> RpcClient::GetConnection(std::shared_ptr<ConnectionId> remote_id) {
+  return cp_->GetConnection(remote_id);
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/rpc-fault-injector.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/rpc-fault-injector.cc b/hbase-native-client/src/hbase/connection/rpc-fault-injector.cc
new file mode 100644
index 0000000..202b21d
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/rpc-fault-injector.cc
@@ -0,0 +1,21 @@
+/*
+ * 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 "hbase/connection/rpc-fault-injector.h"
+
+namespace hbase {} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/rpc-test-server-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/rpc-test-server-handler.cc b/hbase-native-client/src/hbase/connection/rpc-test-server-handler.cc
new file mode 100644
index 0000000..b371ba9
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/rpc-test-server-handler.cc
@@ -0,0 +1,81 @@
+/*
+ * 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 "hbase/connection/rpc-test-server-handler.h"
+#include "hbase/if/RPC.pb.h"
+#include "hbase/if/test.pb.h"
+
+namespace hbase {
+
+void RpcTestServerSerializeHandler::read(Context* ctx, std::unique_ptr<folly::IOBuf> buf) {
+  buf->coalesce();
+  pb::RequestHeader header;
+
+  int used_bytes = serde_.ParseDelimited(buf.get(), &header);
+  VLOG(3) << "Read RPC RequestHeader size=" << used_bytes << " call_id=" << header.call_id();
+
+  auto received = CreateReceivedRequest(header.method_name());
+
+  buf->trimStart(used_bytes);
+  if (header.has_request_param() && received != nullptr) {
+    used_bytes = serde_.ParseDelimited(buf.get(), received->req_msg().get());
+    VLOG(3) << "Read RPCRequest, buf length:" << buf->length()
+            << ", header PB length:" << used_bytes;
+    received->set_call_id(header.call_id());
+  }
+
+  if (received != nullptr) {
+    ctx->fireRead(std::move(received));
+  }
+}
+
+folly::Future<folly::Unit> RpcTestServerSerializeHandler::write(Context* ctx,
+                                                                std::unique_ptr<Response> resp) {
+  VLOG(3) << "Writing RPC Request";
+  // Send the data down the pipeline.
+  return ctx->fireWrite(
+      serde_.Response(resp->call_id(), resp->resp_msg().get(), resp->exception()));
+}
+
+std::unique_ptr<Request> RpcTestServerSerializeHandler::CreateReceivedRequest(
+    const std::string& method_name) {
+  std::unique_ptr<Request> result = nullptr;
+
+  if (method_name == "ping") {
+    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                       std::make_shared<EmptyResponseProto>(), method_name);
+  } else if (method_name == "echo") {
+    result = std::make_unique<Request>(std::make_shared<EchoRequestProto>(),
+                                       std::make_shared<EchoResponseProto>(), method_name);
+  } else if (method_name == "error") {
+    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                       std::make_shared<EmptyResponseProto>(), method_name);
+  } else if (method_name == "pause") {
+    result = std::make_unique<Request>(std::make_shared<PauseRequestProto>(),
+                                       std::make_shared<EmptyResponseProto>(), method_name);
+  } else if (method_name == "addr") {
+    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                       std::make_shared<AddrResponseProto>(), method_name);
+  } else if (method_name == "socketNotOpen") {
+    result = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                       std::make_shared<EmptyResponseProto>(), method_name);
+  }
+  return result;
+}
+}  // end of namespace hbase


[04/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/rpc-test-server.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/rpc-test-server.cc b/hbase-native-client/src/hbase/connection/rpc-test-server.cc
new file mode 100644
index 0000000..337266e
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/rpc-test-server.cc
@@ -0,0 +1,108 @@
+/*
+ * 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 <wangle/channel/AsyncSocketHandler.h>
+#include <wangle/channel/EventBaseHandler.h>
+#include <wangle/codec/LengthFieldBasedFrameDecoder.h>
+#include <wangle/codec/LengthFieldPrepender.h>
+#include <wangle/service/ServerDispatcher.h>
+
+#include "hbase/connection/rpc-test-server-handler.h"
+#include "hbase/connection/rpc-test-server.h"
+#include "hbase/if/test.pb.h"
+
+namespace hbase {
+
+RpcTestServerSerializePipeline::Ptr RpcTestServerPipelineFactory::newPipeline(
+    std::shared_ptr<AsyncTransportWrapper> sock) {
+  if (service_ == nullptr) {
+    initService(sock);
+  }
+  CHECK(service_ != nullptr);
+
+  auto pipeline = RpcTestServerSerializePipeline::create();
+  pipeline->addBack(AsyncSocketHandler(sock));
+  // ensure we can write from any thread
+  pipeline->addBack(EventBaseHandler());
+  pipeline->addBack(LengthFieldBasedFrameDecoder());
+  pipeline->addBack(RpcTestServerSerializeHandler());
+  pipeline->addBack(MultiplexServerDispatcher<std::unique_ptr<Request>, std::unique_ptr<Response>>(
+      service_.get()));
+  pipeline->finalize();
+
+  return pipeline;
+}
+
+void RpcTestServerPipelineFactory::initService(std::shared_ptr<AsyncTransportWrapper> sock) {
+  /* get server address */
+  SocketAddress localAddress;
+  sock->getLocalAddress(&localAddress);
+
+  /* init service with server address */
+  service_ = std::make_shared<ExecutorFilter<std::unique_ptr<Request>, std::unique_ptr<Response>>>(
+      std::make_shared<CPUThreadPoolExecutor>(1),
+      std::make_shared<RpcTestService>(std::make_shared<SocketAddress>(localAddress)));
+}
+
+Future<std::unique_ptr<Response>> RpcTestService::operator()(std::unique_ptr<Request> request) {
+  /* build Response */
+  auto response = std::make_unique<Response>();
+  response->set_call_id(request->call_id());
+  std::string method_name = request->method();
+
+  if (method_name == "ping") {
+    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
+    response->set_resp_msg(pb_resp_msg);
+    VLOG(1) << "RPC server:"
+            << " ping called.";
+
+  } else if (method_name == "echo") {
+    auto pb_resp_msg = std::make_shared<EchoResponseProto>();
+    /* get msg from client */
+    auto pb_req_msg = std::static_pointer_cast<EchoRequestProto>(request->req_msg());
+    pb_resp_msg->set_message(pb_req_msg->message());
+    response->set_resp_msg(pb_resp_msg);
+    VLOG(1) << "RPC server:"
+            << " echo called, " << pb_req_msg->message();
+
+  } else if (method_name == "error") {
+    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
+    response->set_resp_msg(pb_resp_msg);
+    VLOG(1) << "RPC server:"
+            << " error called.";
+    response->set_exception(RpcTestException("server error!"));
+
+  } else if (method_name == "pause") {
+    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
+    /* sleeping */
+    auto pb_req_msg = std::static_pointer_cast<PauseRequestProto>(request->req_msg());
+    std::this_thread::sleep_for(std::chrono::milliseconds(pb_req_msg->ms()));
+    response->set_resp_msg(pb_resp_msg);
+    VLOG(1) << "RPC server:"
+            << " pause called, " << pb_req_msg->ms() << " ms";
+
+  } else if (method_name == "addr") {
+    // TODO:
+  } else if (method_name == "socketNotOpen") {
+    auto pb_resp_msg = std::make_shared<EmptyResponseProto>();
+    response->set_resp_msg(pb_resp_msg);
+  }
+
+  return folly::makeFuture<std::unique_ptr<Response>>(std::move(response));
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/rpc-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/rpc-test.cc b/hbase-native-client/src/hbase/connection/rpc-test.cc
new file mode 100644
index 0000000..64f8f99
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/rpc-test.cc
@@ -0,0 +1,284 @@
+/*
+ * 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 <wangle/bootstrap/ClientBootstrap.h>
+#include <wangle/channel/Handler.h>
+
+#include <folly/Format.h>
+#include <folly/Logging.h>
+#include <folly/SocketAddress.h>
+#include <folly/String.h>
+#include <folly/experimental/TestUtil.h>
+#include <folly/io/async/AsyncSocketException.h>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <boost/thread.hpp>
+#include <chrono>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/if/test.pb.h"
+#include "hbase/connection/rpc-test-server.h"
+#include "hbase/security/user.h"
+#include "hbase/serde/rpc-serde.h"
+
+using namespace wangle;
+using namespace folly;
+using namespace hbase;
+using namespace std::chrono;
+
+DEFINE_int32(port, 0, "test server port");
+DEFINE_string(result_format, "RPC {} returned: {}.", "output format of RPC result");
+DEFINE_string(fail_ex_format, "Shouldn't get here, exception is expected for RPC {}.",
+              "output format of enforcing fail with exception");
+DEFINE_string(fail_no_ex_format, "Shouldn't get here, exception is not expected for RPC {}.",
+              "output format of enforcing fail without exception");
+typedef ServerBootstrap<RpcTestServerSerializePipeline> ServerTestBootstrap;
+typedef std::shared_ptr<ServerTestBootstrap> ServerPtr;
+
+class RpcTest : public ::testing::Test {
+ public:
+  static void SetUpTestCase() { google::InstallFailureSignalHandler(); }
+};
+
+std::shared_ptr<Configuration> CreateConf() {
+  auto conf = std::make_shared<Configuration>();
+  conf->Set(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE, "true");
+  return conf;
+}
+
+ServerPtr CreateRpcServer() {
+  /* create rpc test server */
+  auto server = std::make_shared<ServerTestBootstrap>();
+  server->childPipeline(std::make_shared<RpcTestServerPipelineFactory>());
+  server->bind(FLAGS_port);
+  return server;
+}
+
+std::shared_ptr<folly::SocketAddress> GetRpcServerAddress(ServerPtr server) {
+  auto addr = std::make_shared<folly::SocketAddress>();
+  server->getSockets()[0]->getAddress(addr.get());
+  return addr;
+}
+
+std::shared_ptr<RpcClient> CreateRpcClient(std::shared_ptr<Configuration> conf) {
+  auto io_executor = std::make_shared<wangle::IOThreadPoolExecutor>(1);
+  auto cpu_executor = std::make_shared<wangle::CPUThreadPoolExecutor>(1);
+  auto client = std::make_shared<RpcClient>(io_executor, cpu_executor, nullptr, conf);
+  return client;
+}
+
+std::shared_ptr<RpcClient> CreateRpcClient(std::shared_ptr<Configuration> conf,
+                                           std::chrono::nanoseconds connect_timeout) {
+  auto io_executor = std::make_shared<wangle::IOThreadPoolExecutor>(1);
+  auto cpu_executor = std::make_shared<wangle::CPUThreadPoolExecutor>(1);
+  auto client =
+      std::make_shared<RpcClient>(io_executor, cpu_executor, nullptr, conf, connect_timeout);
+  return client;
+}
+
+/**
+* test ping
+*/
+TEST_F(RpcTest, Ping) {
+  auto conf = CreateConf();
+  auto server = CreateRpcServer();
+  auto server_addr = GetRpcServerAddress(server);
+  auto client = CreateRpcClient(conf);
+
+  auto method = "ping";
+  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                           std::make_shared<EmptyResponseProto>(), method);
+
+  /* sending out request */
+  client
+      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
+                  hbase::security::User::defaultUser())
+      .then([&](std::unique_ptr<Response> response) {
+        auto pb_resp = std::static_pointer_cast<EmptyResponseProto>(response->resp_msg());
+        EXPECT_TRUE(pb_resp != nullptr);
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, "");
+      })
+      .onError([&](const folly::exception_wrapper& ew) {
+        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
+      })
+      .get();
+
+  server->stop();
+  server->join();
+}
+
+/**
+ * test echo
+ */
+TEST_F(RpcTest, Echo) {
+  auto conf = CreateConf();
+  auto server = CreateRpcServer();
+  auto server_addr = GetRpcServerAddress(server);
+  auto client = CreateRpcClient(conf);
+
+  auto method = "echo";
+  auto greetings = "hello, hbase server!";
+  auto request = std::make_unique<Request>(std::make_shared<EchoRequestProto>(),
+                                           std::make_shared<EchoResponseProto>(), method);
+  auto pb_msg = std::static_pointer_cast<EchoRequestProto>(request->req_msg());
+  pb_msg->set_message(greetings);
+
+  /* sending out request */
+  client
+      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
+                  hbase::security::User::defaultUser())
+      .then([&](std::unique_ptr<Response> response) {
+        auto pb_resp = std::static_pointer_cast<EchoResponseProto>(response->resp_msg());
+        EXPECT_TRUE(pb_resp != nullptr);
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, pb_resp->message());
+        EXPECT_EQ(greetings, pb_resp->message());
+      })
+      .onError([&](const folly::exception_wrapper& ew) {
+        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
+      })
+      .get();
+
+  server->stop();
+  server->join();
+}
+
+/**
+ * test error
+ */
+TEST_F(RpcTest, Error) {
+  auto conf = CreateConf();
+  auto server = CreateRpcServer();
+  auto server_addr = GetRpcServerAddress(server);
+  auto client = CreateRpcClient(conf);
+
+  auto method = "error";
+  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                           std::make_shared<EmptyResponseProto>(), method);
+  /* sending out request */
+  client
+      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
+                  hbase::security::User::defaultUser())
+      .then([&](std::unique_ptr<Response> response) {
+        FAIL() << folly::sformat(FLAGS_fail_ex_format, method);
+      })
+      .onError([&](const folly::exception_wrapper& ew) {
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
+        std::string kRemoteException = demangle(typeid(hbase::RemoteException)).toStdString();
+        std::string kRpcTestException = demangle(typeid(hbase::RpcTestException)).toStdString();
+
+        /* verify exception_wrapper */
+        EXPECT_TRUE(bool(ew));
+        EXPECT_EQ(kRemoteException, ew.class_name());
+
+        /* verify exception */
+        EXPECT_TRUE(ew.with_exception([&](const hbase::RemoteException& e) {
+          EXPECT_EQ(kRpcTestException, e.exception_class_name());
+          EXPECT_EQ(kRpcTestException + ": server error!", e.stack_trace());
+        }));
+      })
+      .get();
+
+  server->stop();
+  server->join();
+}
+
+TEST_F(RpcTest, SocketNotOpen) {
+  auto conf = CreateConf();
+  auto server = CreateRpcServer();
+  auto server_addr = GetRpcServerAddress(server);
+  auto client = CreateRpcClient(conf);
+
+  auto method = "socketNotOpen";
+  auto request = std::make_unique<Request>(std::make_shared<EmptyRequestProto>(),
+                                           std::make_shared<EmptyResponseProto>(), method);
+
+  server->stop();
+  server->join();
+
+  /* sending out request */
+  client
+      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
+                  hbase::security::User::defaultUser())
+      .then([&](std::unique_ptr<Response> response) {
+        FAIL() << folly::sformat(FLAGS_fail_ex_format, method);
+      })
+      .onError([&](const folly::exception_wrapper& ew) {
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
+        std::string kConnectionException =
+            demangle(typeid(hbase::ConnectionException)).toStdString();
+        std::string kAsyncSocketException =
+            demangle(typeid(folly::AsyncSocketException)).toStdString();
+
+        /* verify exception_wrapper */
+        EXPECT_TRUE(bool(ew));
+        EXPECT_EQ(kConnectionException, ew.class_name());
+
+        /* verify exception */
+        EXPECT_TRUE(ew.with_exception([&](const hbase::ConnectionException& e) {
+          EXPECT_TRUE(bool(e.cause()));
+          EXPECT_EQ(kAsyncSocketException, e.cause().class_name());
+          VLOG(1) << folly::sformat(FLAGS_result_format, method, e.cause().what());
+          e.cause().with_exception([&](const folly::AsyncSocketException& ase) {
+            EXPECT_EQ(AsyncSocketException::AsyncSocketExceptionType::NOT_OPEN, ase.getType());
+            EXPECT_EQ(111 /*ECONNREFUSED*/, ase.getErrno());
+          });
+        }));
+      })
+      .get();
+}
+
+/**
+ * test pause
+ */
+TEST_F(RpcTest, Pause) {
+  int ms = 500;
+
+  auto conf = CreateConf();
+  auto server = CreateRpcServer();
+  auto server_addr = GetRpcServerAddress(server);
+  auto client =
+      CreateRpcClient(conf, std::chrono::duration_cast<nanoseconds>(milliseconds(2 * ms)));
+
+  auto method = "pause";
+  auto request = std::make_unique<Request>(std::make_shared<PauseRequestProto>(),
+                                           std::make_shared<EmptyResponseProto>(), method);
+  auto pb_msg = std::static_pointer_cast<PauseRequestProto>(request->req_msg());
+
+  pb_msg->set_ms(ms);
+
+  /* sending out request */
+  client
+      ->AsyncCall(server_addr->getAddressStr(), server_addr->getPort(), std::move(request),
+                  hbase::security::User::defaultUser())
+      .then([&](std::unique_ptr<Response> response) {
+        auto pb_resp = std::static_pointer_cast<EmptyResponseProto>(response->resp_msg());
+        EXPECT_TRUE(pb_resp != nullptr);
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, "");
+      })
+      .onError([&](const folly::exception_wrapper& ew) {
+        VLOG(1) << folly::sformat(FLAGS_result_format, method, ew.what());
+        FAIL() << folly::sformat(FLAGS_fail_no_ex_format, method);
+      })
+      .get();
+
+  server->stop();
+  server->join();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/sasl-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/sasl-handler.cc b/hbase-native-client/src/hbase/connection/sasl-handler.cc
new file mode 100644
index 0000000..242665f
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/sasl-handler.cc
@@ -0,0 +1,225 @@
+/*
+ * 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 "hbase/connection/sasl-handler.h"
+
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+#include <sasl/saslutil.h>
+
+#include <google/protobuf/io/coded_stream.h>
+#include <google/protobuf/io/zero_copy_stream_impl.h>
+#include <wangle/channel/Handler.h>
+
+#include <condition_variable>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <utility>
+
+#include "hbase/connection/service.h"
+#include "hbase/security/user.h"
+using hbase::security::User;
+
+using std::chrono::nanoseconds;
+using namespace folly;
+using namespace wangle;
+using namespace hbase;
+
+SaslHandler::SaslHandler(std::string user_name, std::shared_ptr<Configuration> conf)
+    : user_name_(user_name) {
+  host_name_.clear();
+  secure_ = User::IsSecurityEnabled(*conf);
+  service_name_ = SaslUtil::ParseServiceName(conf, secure_);
+  sasl_connection_setup_started_.clear();
+  sasl_connection_setup_in_progress_.store(true);
+}
+
+SaslHandler::SaslHandler(const SaslHandler &hdlr) {
+  user_name_ = hdlr.user_name_;
+  service_name_ = hdlr.service_name_;
+  secure_ = hdlr.secure_;
+  host_name_ = hdlr.host_name_;
+  // copy-constructor sets the flags below to their initial state as opposed to getting them
+  // from the object this class is constructed from. That way, this instance is ready to do
+  // sasl stuff without issues, right from the SaslInit. Sharing a sasl session is not useful
+  // between two handler instances.
+  sasl_connection_setup_started_.clear();
+  sasl_connection_setup_in_progress_.store(true);
+  sconn_ = nullptr;
+}
+
+SaslHandler::~SaslHandler() {
+  if (nullptr != sconn_) {
+    sasl_dispose(&sconn_);
+  }
+  sconn_ = nullptr;
+}
+
+void SaslHandler::transportActive(Context *ctx) {
+  // assign hostname; needed for the sasl handshake if secure
+  folly::SocketAddress address;
+  ctx->getTransport()->getPeerAddress(&address);
+  host_name_ = address.getHostStr();
+
+  // now init the sasl library; this is once per process
+  if (secure_) {
+    sasl_util_.InitializeSaslLib();
+  }
+  // write the preamble to kick off the RPC handshake
+  VLOG(3) << "Writing RPC connection Preamble to server: " << host_name_;
+  auto preamble = RpcSerde::Preamble(secure_);
+  ctx->fireWrite(std::move(preamble));
+  ctx->fireTransportActive();
+}
+
+void SaslHandler::read(Context *ctx, folly::IOBufQueue &buf) {
+  // if security is not on, or in case of security-on, if secure connection setup not in progress,
+  // pass it up without touching
+  if (!secure_ || !sasl_connection_setup_in_progress_.load()) {
+    ctx->fireRead(buf);
+  } else {
+    // message is for this handler; process it appropriately
+    ContinueSaslNegotiation(ctx, &buf);
+  }
+}
+
+folly::Future<folly::Unit> SaslHandler::write(Context *ctx, std::unique_ptr<folly::IOBuf> buf) {
+  // if security is on, and if secure connection setup in progress,
+  // this message is for this handler to process and respond
+  if (secure_ && sasl_connection_setup_in_progress_.load()) {
+    // store IOBuf which is to be sent to server after SASL handshake
+    iobuf_.push_back(std::move(buf));
+    if (!sasl_connection_setup_started_.test_and_set()) {
+      // for the first incoming RPC from the higher layer, trigger sasl initialization
+      return SaslInit(ctx);
+    } else {
+      // for the subsequent incoming RPCs from the higher layer, just return empty future
+      folly::Promise<folly::Unit> p_;
+      return p_.getFuture();
+    }
+  }
+  // pass the bytes recieved down without touching it
+  return ctx->fireWrite(std::move(buf));
+}
+
+folly::Future<folly::Unit> SaslHandler::WriteSaslOutput(Context *ctx, const char *out,
+                                                        unsigned int outlen) {
+  int buffer_size = outlen + 4;
+  auto iob = IOBuf::create(buffer_size);
+  iob->append(buffer_size);
+  // Create the array output stream.
+  google::protobuf::io::ArrayOutputStream aos{iob->writableData(), buffer_size};
+  std::unique_ptr<google::protobuf::io::CodedOutputStream> coded_output =
+      std::make_unique<google::protobuf::io::CodedOutputStream>(&aos);
+  uint32_t total_size = outlen;
+  total_size = ntohl(total_size);
+  coded_output->WriteRaw(&total_size, 4);
+  coded_output->WriteRaw(out, outlen);
+  return ctx->fireWrite(std::move(iob));
+}
+
+void SaslHandler::FinishAuth(Context *ctx, folly::IOBufQueue *bufQueue) {
+  std::unique_ptr<folly::IOBuf> iob;
+  if (!bufQueue->empty()) {
+    iob = bufQueue->pop_front();
+    throw std::runtime_error("Error in the final step of handshake " +
+                             std::string(reinterpret_cast<const char *>(iob->data())));
+  } else {
+    sasl_connection_setup_in_progress_.store(false);
+    // write what we buffered
+    for (size_t i = 0; i < iobuf_.size(); i++) {
+      iob = std::move(iobuf_.at(i));
+      ctx->fireWrite(std::move(iob));
+    }
+  }
+}
+
+folly::Future<folly::Unit> SaslHandler::SaslInit(Context *ctx) {
+  int rc;
+  const char *mechusing, *mechlist = "GSSAPI";
+  const char *out;
+  unsigned int outlen;
+
+  rc = sasl_client_new(service_name_.c_str(), /* The service we are using*/
+                       host_name_.c_str(), NULL,
+                       NULL, /* Local and remote IP address strings
+                                   (NULL disables mechanisms which require this info)*/
+                       NULL, /*connection-specific callbacks*/
+                       0 /*security flags*/, &sconn_);
+  if (rc != SASL_OK) {
+    LOG(FATAL) << "Cannot create client (" << rc << ") ";
+    throw std::runtime_error("Cannot create client");
+  }
+  int curr_rc;
+  do {
+    curr_rc = sasl_client_start(sconn_,   /* the same context from above */
+                                mechlist, /* the list of mechanisms from the server */
+                                NULL,     /* filled in if an interaction is needed */
+                                &out,     /* filled in on success */
+                                &outlen,  /* filled in on success */
+                                &mechusing);
+  } while (curr_rc == SASL_INTERACT); /* the mechanism may ask us to fill
+     in things many times. result is SASL_CONTINUE on success */
+  if (curr_rc != SASL_CONTINUE) {
+    throw std::runtime_error("Cannot start client (" + std::to_string(curr_rc) + ")");
+  }
+  folly::Future<folly::Unit> fut = WriteSaslOutput(ctx, out, outlen);
+  return fut;
+}
+
+void SaslHandler::ContinueSaslNegotiation(Context *ctx, folly::IOBufQueue *bufQueue) {
+  const char *out;
+  unsigned int outlen;
+
+  int bytes_sent = 0;
+  int bytes_received = 0;
+
+  std::unique_ptr<folly::IOBuf> iob = bufQueue->pop_front();
+  bytes_received = iob->length();
+  if (bytes_received == 0) {
+    throw std::runtime_error("Error in sasl handshake");
+  }
+  folly::io::RWPrivateCursor c(iob.get());
+  std::uint32_t status = c.readBE<std::uint32_t>();
+  std::uint32_t sz = c.readBE<std::uint32_t>();
+
+  if (status != 0 /*Status 0 is success*/) {
+    // Assumption here is that the response from server is not more than 8 * 1024
+    throw std::runtime_error("Error in sasl handshake " +
+                             std::string(reinterpret_cast<char *>(c.writableData())));
+  }
+  out = nullptr;
+  outlen = 0;
+
+  int curr_rc =
+      sasl_client_step(sconn_,                                     /* our context */
+                       reinterpret_cast<char *>(c.writableData()), /* the data from the server */
+                       sz,                                         /* its length */
+                       NULL,     /* this should be unallocated and NULL */
+                       &out,     /* filled in on success */
+                       &outlen); /* filled in on success */
+
+  if (curr_rc == SASL_OK || curr_rc == SASL_CONTINUE) {
+    WriteSaslOutput(ctx, out, outlen);
+  }
+  if (curr_rc == SASL_OK) {
+    FinishAuth(ctx, bufQueue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/connection/sasl-util.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/connection/sasl-util.cc b/hbase-native-client/src/hbase/connection/sasl-util.cc
new file mode 100644
index 0000000..7e7403e
--- /dev/null
+++ b/hbase-native-client/src/hbase/connection/sasl-util.cc
@@ -0,0 +1,92 @@
+/*
+ * 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 "hbase/connection/sasl-util.h"
+
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+#include <sasl/saslutil.h>
+
+#include <string>
+
+int SaslUtil::GetPluginPath(void *context __attribute__((unused)), const char **path) {
+  *path = getenv("SASL_PATH");
+
+  if (*path == NULL) {
+    *path = kDefaultPluginDir;
+  }
+  return SASL_OK;
+}
+
+void *SaslUtil::MutexNew(void) {
+  auto m = new std::mutex();
+  return m;
+}
+
+int SaslUtil::MutexLock(void *m) {
+  (reinterpret_cast<std::mutex *>(m))->lock();
+  return SASL_OK;
+}
+
+int SaslUtil::MutexUnlock(void *m) {
+  (reinterpret_cast<std::mutex *>(m))->unlock();
+  return SASL_OK;
+}
+
+void SaslUtil::MutexDispose(void *m) {
+  std::mutex *mutex = reinterpret_cast<std::mutex *>(m);
+  delete mutex;
+}
+
+std::once_flag SaslUtil::library_inited_;
+
+void SaslUtil::InitializeSaslLib() {
+  std::call_once(library_inited_, []() {
+    sasl_set_mutex(reinterpret_cast<sasl_mutex_alloc_t *>(&SaslUtil::MutexNew),
+                   reinterpret_cast<sasl_mutex_lock_t *>(&SaslUtil::MutexLock),
+                   reinterpret_cast<sasl_mutex_unlock_t *>(&SaslUtil::MutexUnlock),
+                   reinterpret_cast<sasl_mutex_free_t *>(&SaslUtil::MutexDispose));
+    static sasl_callback_t callbacks[] = {
+        {SASL_CB_GETPATH, (sasl_callback_ft)&SaslUtil::GetPluginPath, NULL},
+        {SASL_CB_LIST_END, NULL, NULL}};
+    int rc = sasl_client_init(callbacks);
+    if (rc != SASL_OK) {
+      throw std::runtime_error("Cannot initialize client " + std::to_string(rc));
+    }
+  });
+}
+
+std::string SaslUtil::ParseServiceName(std::shared_ptr<hbase::Configuration> conf, bool secure) {
+  if (!secure) {
+    return std::string();
+  }
+  std::string svrPrincipal = conf->Get(kServerPrincipalConfKey, "");
+  // principal is of this form: hbase/23a03935850c@EXAMPLE.COM
+  // where 23a03935850c is the host (optional)
+  std::size_t pos = svrPrincipal.find("/");
+  if (pos == std::string::npos && svrPrincipal.find("@") != std::string::npos) {
+    pos = svrPrincipal.find("@");
+  }
+  if (pos == std::string::npos) {
+    throw std::runtime_error("Couldn't retrieve service principal from conf");
+  }
+  VLOG(1) << "pos " << pos << " " << svrPrincipal;
+  std::string service_name = svrPrincipal.substr(0, pos);
+  return service_name;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/exceptions/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/exceptions/BUCK b/hbase-native-client/src/hbase/exceptions/BUCK
new file mode 100644
index 0000000..00ed344
--- /dev/null
+++ b/hbase-native-client/src/hbase/exceptions/BUCK
@@ -0,0 +1,37 @@
+##
+# 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.
+
+cxx_library(
+    name="exceptions",
+    srcs=[
+        "exception.cc",
+    ],
+    deps=[
+        "//include/hbase/exceptions:exceptions",
+        "//third-party:folly",
+    ],
+    compiler_flags=['-Weffc++'],
+    visibility=['//src/hbase/client/...', '//src/hbase/connection/...'],)
+cxx_test(
+    name="exception-test",
+    srcs=[
+        "exception-test.cc",
+    ],
+    deps=[
+        ":exceptions",
+    ],
+    run_test_separately=True,)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/exceptions/exception-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/exceptions/exception-test.cc b/hbase-native-client/src/hbase/exceptions/exception-test.cc
new file mode 100644
index 0000000..e28f084
--- /dev/null
+++ b/hbase-native-client/src/hbase/exceptions/exception-test.cc
@@ -0,0 +1,64 @@
+/*
+ * 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 <gtest/gtest.h>
+
+#include "hbase/exceptions/exception.h"
+
+#include "folly/ExceptionWrapper.h"
+
+using hbase::ExceptionUtil;
+using hbase::IOException;
+using hbase::RemoteException;
+
+TEST(ExceptionUtilTest, IOExceptionShouldRetry) {
+  IOException ex{};
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_do_not_retry(true);
+  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_do_not_retry(false);
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  IOException ex2{"description", true};
+  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex2));
+
+  IOException ex3{"description", std::runtime_error("ex"), true};
+  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex3));
+}
+
+TEST(ExceptionUtilTest, RemoteExceptionShouldRetry) {
+  RemoteException ex{};
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_do_not_retry(true);
+  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_do_not_retry(false);
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_exception_class_name("org.apache.hadoop.hbase.FooException");
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_exception_class_name("org.apache.hadoop.hbase.NotServingRegionException");
+  EXPECT_TRUE(ExceptionUtil::ShouldRetry(ex));
+
+  ex.set_exception_class_name("org.apache.hadoop.hbase.UnknownRegionException");
+  EXPECT_FALSE(ExceptionUtil::ShouldRetry(ex));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/exceptions/exception.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/exceptions/exception.cc b/hbase-native-client/src/hbase/exceptions/exception.cc
new file mode 100644
index 0000000..91ff818
--- /dev/null
+++ b/hbase-native-client/src/hbase/exceptions/exception.cc
@@ -0,0 +1,128 @@
+/*
+ * 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 "hbase/exceptions/exception.h"
+
+namespace hbase {
+const std::vector<const char*> ExceptionUtil::kAllDoNotRetryIOExceptions = {
+    kDoNotRetryIOException,
+    kTableNotFoundException,
+    kTableNotEnabledException,
+    kCoprocessorException,
+    kBypassCoprocessorException,
+    kInvalidFamilyOperationException,
+    kServerTooBusyException,
+    kFailedSanityCheckException,
+    kCorruptHFileException,
+    kLabelAlreadyExistsException,
+    kFatalConnectionException,
+    kUnsupportedCryptoException,
+    kUnsupportedCellCodecException,
+    kEmptyServiceNameException,
+    kUnknownServiceException,
+    kWrongVersionException,
+    kBadAuthException,
+    kUnsupportedCompressionCodecException,
+    kDoNotRetryRegionException,
+    kRowTooBigException,
+    kRowTooBigExceptionDeprecated,
+    kUnknownRegionException,
+    kMergeRegionException,
+    kNoServerForRegionException,
+    kQuotaExceededException,
+    kSpaceLimitingException,
+    kThrottlingException,
+    kAccessDeniedException,
+    kUnknownProtocolException,
+    kRequestTooBigException,
+    kNotAllMetaRegionsOnlineException,
+    kConstraintException,
+    kNoSuchColumnFamilyException,
+    kLeaseException,
+    kInvalidLabelException,
+    kUnknownScannerException,
+    kScannerResetException,
+    kOutOfOrderScannerNextException};
+
+bool ExceptionUtil::ShouldRetry(const folly::exception_wrapper& error) {
+  bool do_not_retry = false;
+  error.with_exception(
+      [&](const IOException& ioe) { do_not_retry = do_not_retry || ioe.do_not_retry(); });
+  error.with_exception([&](const RemoteException& remote_ex) {
+    do_not_retry = do_not_retry || IsJavaDoNotRetryException(remote_ex.exception_class_name());
+  });
+  return !do_not_retry;
+}
+
+/**
+ * Returns whether the java exception class extends DoNotRetryException.
+ * In the java side, we just have a hierarchy of Exception classes that we use
+ * both client side and server side. On the client side, we rethrow the server
+ * side exception by un-wrapping the exception from a RemoteException or a ServiceException
+ * (see ConnectionUtils.translateException() in Java).
+ * Since this object-hierarchy info is not available in C++ side, we are doing a
+ * very fragile catch-all list of all exception types in Java that extend the
+ * DoNotRetryException class type.
+ */
+bool ExceptionUtil::IsJavaDoNotRetryException(const std::string& java_class_name) {
+  for (auto exception : kAllDoNotRetryIOExceptions) {
+    if (java_class_name == exception) {
+      return true;
+    }
+  }
+  return false;
+}
+
+/**
+ * Returns whether the scanner is closed when the client received the
+ * remote exception.
+ * Since the object-hierarchy info is not available in C++ side, we are doing a
+ * very fragile catch-all list of all exception types in Java that extend these
+ * three base classes: UnknownScannerException, NotServingRegionException,
+ * RegionServerStoppedException
+ */
+bool ExceptionUtil::IsScannerClosed(const folly::exception_wrapper& exception) {
+  bool scanner_closed = false;
+  exception.with_exception([&](const RemoteException& remote_ex) {
+    auto java_class = remote_ex.exception_class_name();
+    if (java_class == kUnknownScannerException || java_class == kNotServingRegionException ||
+        java_class == kRegionInRecoveryException || java_class == kRegionOpeningException ||
+        java_class == kRegionMovedException || java_class == kRegionServerStoppedException ||
+        java_class == kRegionServerAbortedException) {
+      scanner_closed = true;
+    }
+  });
+  return scanner_closed;
+}
+
+/**
+ * Returns whether the wrapped exception is a java exception of type OutOfOrderScannerNextException
+ * or ScannerResetException. These two exception types are thrown from the server side when the
+ * scanner on the server side is closed.
+ */
+bool ExceptionUtil::IsScannerOutOfOrder(const folly::exception_wrapper& exception) {
+  bool scanner_out_of_order = false;
+  exception.with_exception([&](const RemoteException& remote_ex) {
+    auto java_class = remote_ex.exception_class_name();
+    if (java_class == kOutOfOrderScannerNextException || java_class == kScannerResetException) {
+      scanner_out_of_order = true;
+    }
+  });
+  return scanner_out_of_order;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/AccessControl.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/AccessControl.proto b/hbase-native-client/src/hbase/if/AccessControl.proto
new file mode 100644
index 0000000..e67540b
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/AccessControl.proto
@@ -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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "AccessControlProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+message Permission {
+    enum Action {
+        READ = 0;
+        WRITE = 1;
+        EXEC = 2;
+        CREATE = 3;
+        ADMIN = 4;
+    }
+    enum Type {
+        Global = 1;
+        Namespace = 2;
+        Table = 3;
+    }
+    required Type type = 1;
+    optional GlobalPermission global_permission = 2;
+    optional NamespacePermission namespace_permission = 3;
+    optional TablePermission table_permission = 4;
+}
+
+message TablePermission {
+    optional TableName table_name = 1;
+    optional bytes family = 2;
+    optional bytes qualifier = 3;
+    repeated Permission.Action action = 4;
+}
+
+message NamespacePermission {
+    optional bytes namespace_name = 1;
+    repeated Permission.Action action = 2;
+}
+
+message GlobalPermission {
+    repeated Permission.Action action = 1;
+}
+
+message UserPermission {
+    required bytes user = 1;
+    required Permission permission = 3;
+}
+
+/**
+ * Content of the /hbase/acl/<table or namespace> znode.
+ */
+message UsersAndPermissions {
+  message UserPermissions {
+    required bytes user = 1;
+    repeated Permission permissions = 2;
+  }
+
+  repeated UserPermissions user_permissions = 1;
+}
+
+message GrantRequest {
+  required UserPermission user_permission = 1;
+}
+
+message GrantResponse {
+}
+
+message RevokeRequest {
+  required UserPermission user_permission = 1;
+}
+
+message RevokeResponse {
+}
+
+message GetUserPermissionsRequest {
+  optional Permission.Type type = 1;
+  optional TableName table_name = 2;
+  optional bytes namespace_name = 3;
+}
+
+message GetUserPermissionsResponse {
+  repeated UserPermission user_permission = 1;
+}
+
+message CheckPermissionsRequest {
+  repeated Permission permission = 1;
+}
+
+message CheckPermissionsResponse {
+}
+
+service AccessControlService {
+    rpc Grant(GrantRequest)
+      returns (GrantResponse);
+
+    rpc Revoke(RevokeRequest)
+      returns (RevokeResponse);
+
+    rpc GetUserPermissions(GetUserPermissionsRequest)
+      returns (GetUserPermissionsResponse);
+
+    rpc CheckPermissions(CheckPermissionsRequest)
+      returns (CheckPermissionsResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Admin.proto b/hbase-native-client/src/hbase/if/Admin.proto
new file mode 100644
index 0000000..a1905a4
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Admin.proto
@@ -0,0 +1,310 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for Admin service.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "AdminProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "Client.proto";
+import "HBase.proto";
+import "WAL.proto";
+
+message GetRegionInfoRequest {
+  required RegionSpecifier region = 1;
+  optional bool compaction_state = 2;
+}
+
+message GetRegionInfoResponse {
+  required RegionInfo region_info = 1;
+  optional CompactionState compaction_state = 2;
+  optional bool isRecovering = 3;
+
+  enum CompactionState {
+    NONE = 0;
+    MINOR = 1;
+    MAJOR = 2;
+    MAJOR_AND_MINOR = 3;
+  }
+}
+
+/**
+ * Get a list of store files for a set of column families in a particular region.
+ * If no column family is specified, get the store files for all column families.
+ */
+message GetStoreFileRequest {
+  required RegionSpecifier region = 1;
+  repeated bytes family = 2;
+}
+
+message GetStoreFileResponse {
+  repeated string store_file = 1;
+}
+
+message GetOnlineRegionRequest {
+}
+
+message GetOnlineRegionResponse {
+  repeated RegionInfo region_info = 1;
+}
+
+message OpenRegionRequest {
+  repeated RegionOpenInfo open_info = 1;
+  // the intended server for this RPC.
+  optional uint64 serverStartCode = 2;
+  // wall clock time from master
+  optional uint64 master_system_time = 5;
+
+  message RegionOpenInfo {
+    required RegionInfo region = 1;
+    optional uint32 version_of_offline_node = 2;
+    repeated ServerName favored_nodes = 3;
+    // open region for distributedLogReplay
+    optional bool openForDistributedLogReplay = 4;
+  }
+}
+
+message OpenRegionResponse {
+  repeated RegionOpeningState opening_state = 1;
+
+  enum RegionOpeningState {
+    OPENED = 0;
+    ALREADY_OPENED = 1;
+    FAILED_OPENING = 2;
+  }
+}
+
+message WarmupRegionRequest {
+
+    required RegionInfo regionInfo = 1;
+}
+
+message WarmupRegionResponse {
+}
+
+/**
+ * Closes the specified region and will use or not use ZK during the close
+ * according to the specified flag.
+ */
+message CloseRegionRequest {
+  required RegionSpecifier region = 1;
+  optional uint32 version_of_closing_node = 2;
+  optional bool transition_in_ZK = 3 [default = true];
+  optional ServerName destination_server = 4;
+  // the intended server for this RPC.
+  optional uint64 serverStartCode = 5;
+}
+
+message CloseRegionResponse {
+  required bool closed = 1;
+}
+
+/**
+ * Flushes the MemStore of the specified region.
+ * <p>
+ * This method is synchronous.
+ */
+message FlushRegionRequest {
+  required RegionSpecifier region = 1;
+  optional uint64 if_older_than_ts = 2;
+  optional bool write_flush_wal_marker = 3; // whether to write a marker to WAL even if not flushed
+}
+
+message FlushRegionResponse {
+  required uint64 last_flush_time = 1;
+  optional bool flushed = 2;
+  optional bool wrote_flush_wal_marker = 3;
+}
+
+/**
+ * Splits the specified region.
+ * <p>
+ * This method currently flushes the region and then forces a compaction which
+ * will then trigger a split.  The flush is done synchronously but the
+ * compaction is asynchronous.
+ */
+message SplitRegionRequest {
+  required RegionSpecifier region = 1;
+  optional bytes split_point = 2;
+}
+
+message SplitRegionResponse {
+}
+
+/**
+ * Compacts the specified region.  Performs a major compaction if specified.
+ * <p>
+ * This method is asynchronous.
+ */
+message CompactRegionRequest {
+  required RegionSpecifier region = 1;
+  optional bool major = 2;
+  optional bytes family = 3;
+}
+
+message CompactRegionResponse {
+}
+
+message UpdateFavoredNodesRequest {
+  repeated RegionUpdateInfo update_info = 1;
+
+  message RegionUpdateInfo {
+    required RegionInfo region = 1;
+    repeated ServerName favored_nodes = 2;
+  }
+}
+
+message UpdateFavoredNodesResponse {
+  optional uint32 response = 1;
+}
+
+/**
+ * Merges the specified regions.
+ * <p>
+ * This method currently closes the regions and then merges them
+ */
+message MergeRegionsRequest {
+  required RegionSpecifier region_a = 1;
+  required RegionSpecifier region_b = 2;
+  optional bool forcible = 3 [default = false];
+  // wall clock time from master
+  optional uint64 master_system_time = 4;
+}
+
+message MergeRegionsResponse {
+}
+
+// Protocol buffer version of WAL for replication
+message WALEntry {
+  required WALKey key = 1;
+  // Following may be null if the KVs/Cells are carried along the side in a cellblock (See
+  // RPC for more on cellblocks). If Cells/KVs are in a cellblock, this next field is null
+  // and associated_cell_count has count of Cells associated w/ this WALEntry
+  repeated bytes key_value_bytes = 2;
+  // If Cell data is carried alongside in a cellblock, this is count of Cells in the cellblock.
+  optional int32 associated_cell_count = 3;
+}
+
+/**
+ * Replicates the given entries. The guarantee is that the given entries
+ * will be durable on the slave cluster if this method returns without
+ * any exception.  hbase.replication has to be set to true for this to work.
+ */
+message ReplicateWALEntryRequest {
+  repeated WALEntry entry = 1;
+  optional string replicationClusterId = 2;
+  optional string sourceBaseNamespaceDirPath = 3;
+  optional string sourceHFileArchiveDirPath = 4;
+}
+
+message ReplicateWALEntryResponse {
+}
+
+message RollWALWriterRequest {
+}
+
+/*
+ * Roll request responses no longer include regions to flush
+ * this list will always be empty when talking to a 1.0 server
+ */
+message RollWALWriterResponse {
+  // A list of encoded name of regions to flush
+  repeated bytes region_to_flush = 1;
+}
+
+message StopServerRequest {
+  required string reason = 1;
+}
+
+message StopServerResponse {
+}
+
+message GetServerInfoRequest {
+}
+
+message ServerInfo {
+  required ServerName server_name = 1;
+  optional uint32 webui_port = 2;
+}
+
+message GetServerInfoResponse {
+  required ServerInfo server_info = 1;
+}
+
+message UpdateConfigurationRequest {
+}
+
+message UpdateConfigurationResponse {
+}
+
+service AdminService {
+  rpc GetRegionInfo(GetRegionInfoRequest)
+    returns(GetRegionInfoResponse);
+
+  rpc GetStoreFile(GetStoreFileRequest)
+    returns(GetStoreFileResponse);
+
+  rpc GetOnlineRegion(GetOnlineRegionRequest)
+    returns(GetOnlineRegionResponse);
+
+  rpc OpenRegion(OpenRegionRequest)
+    returns(OpenRegionResponse);
+
+  rpc WarmupRegion(WarmupRegionRequest)
+    returns(WarmupRegionResponse);
+
+  rpc CloseRegion(CloseRegionRequest)
+    returns(CloseRegionResponse);
+
+  rpc FlushRegion(FlushRegionRequest)
+    returns(FlushRegionResponse);
+
+  rpc SplitRegion(SplitRegionRequest)
+    returns(SplitRegionResponse);
+
+  rpc CompactRegion(CompactRegionRequest)
+    returns(CompactRegionResponse);
+
+  rpc MergeRegions(MergeRegionsRequest)
+    returns(MergeRegionsResponse);
+
+  rpc ReplicateWALEntry(ReplicateWALEntryRequest)
+    returns(ReplicateWALEntryResponse);
+
+  rpc Replay(ReplicateWALEntryRequest)
+    returns(ReplicateWALEntryResponse);
+
+  rpc RollWALWriter(RollWALWriterRequest)
+    returns(RollWALWriterResponse);
+
+  rpc GetServerInfo(GetServerInfoRequest)
+    returns(GetServerInfoResponse);
+
+  rpc StopServer(StopServerRequest)
+    returns(StopServerResponse);
+
+  rpc UpdateFavoredNodes(UpdateFavoredNodesRequest)
+    returns(UpdateFavoredNodesResponse);
+
+  rpc UpdateConfiguration(UpdateConfigurationRequest)
+    returns(UpdateConfigurationResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Aggregate.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Aggregate.proto b/hbase-native-client/src/hbase/if/Aggregate.proto
new file mode 100644
index 0000000..4d32e70
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Aggregate.proto
@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "AggregateProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "Client.proto";
+
+message AggregateRequest {
+  /** The request passed to the AggregateService consists of three parts
+   *  (1) the (canonical) classname of the ColumnInterpreter implementation
+   *  (2) the Scan query
+   *  (3) any bytes required to construct the ColumnInterpreter object
+   *      properly
+   */
+  required string interpreter_class_name = 1;
+  required Scan scan = 2;
+  optional bytes  interpreter_specific_bytes = 3;
+}
+
+message AggregateResponse {
+  /**
+   * The AggregateService methods all have a response that either is a Pair
+   * or a simple object. When it is a Pair both first_part and second_part
+   * have defined values (and the second_part is not present in the response
+   * when the response is not a pair). Refer to the AggregateImplementation 
+   * class for an overview of the AggregateResponse object constructions. 
+   */ 
+  repeated bytes first_part = 1;
+  optional bytes second_part = 2;
+}
+
+/** Refer to the AggregateImplementation class for an overview of the 
+ *  AggregateService method implementations and their functionality.
+ */
+service AggregateService {
+  rpc GetMax (AggregateRequest) returns (AggregateResponse);
+  rpc GetMin (AggregateRequest) returns (AggregateResponse);
+  rpc GetSum (AggregateRequest) returns (AggregateResponse);
+  rpc GetRowNum (AggregateRequest) returns (AggregateResponse);
+  rpc GetAvg (AggregateRequest) returns (AggregateResponse);
+  rpc GetStd (AggregateRequest) returns (AggregateResponse);
+  rpc GetMedian (AggregateRequest) returns (AggregateResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Authentication.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Authentication.proto b/hbase-native-client/src/hbase/if/Authentication.proto
new file mode 100644
index 0000000..2f64799
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Authentication.proto
@@ -0,0 +1,82 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "AuthenticationProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message AuthenticationKey {
+    required int32 id = 1;
+    required int64 expiration_date = 2;
+    required bytes key = 3;
+}
+
+
+message TokenIdentifier {
+    enum Kind {
+        HBASE_AUTH_TOKEN = 0;
+    }
+    required Kind kind = 1;
+    required bytes username = 2;
+    required int32 key_id = 3;
+    optional int64 issue_date = 4;
+    optional int64 expiration_date = 5;
+    optional int64 sequence_number = 6;
+}
+
+
+// Serialization of the org.apache.hadoop.security.token.Token class
+// Note that this is a Hadoop class, so fields may change!
+message Token {
+    // the TokenIdentifier in serialized form
+    // Note: we can't use the protobuf directly because the Hadoop Token class
+    // only stores the serialized bytes
+    optional bytes identifier = 1;
+    optional bytes password = 2;
+    optional bytes service = 3;
+}
+
+
+// RPC request & response messages
+message GetAuthenticationTokenRequest {
+}
+
+message GetAuthenticationTokenResponse {
+    optional Token token = 1;
+}
+
+message WhoAmIRequest {
+}
+
+message WhoAmIResponse {
+    optional string username = 1;
+    optional string auth_method = 2;
+}
+
+
+// RPC service
+service AuthenticationService {
+    rpc GetAuthenticationToken(GetAuthenticationTokenRequest)
+        returns (GetAuthenticationTokenResponse);
+
+    rpc WhoAmI(WhoAmIRequest)
+        returns (WhoAmIResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/BUCK b/hbase-native-client/src/hbase/if/BUCK
new file mode 100644
index 0000000..c8d51f2
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/BUCK
@@ -0,0 +1,49 @@
+##
+# 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.
+
+PROTO_SRCS = glob(['*.proto'])
+HEADER_FILENAMES = [x.replace('.proto', '.pb.h') for x in PROTO_SRCS]
+CC_FILENAMES = [x.replace('.proto', '.pb.cc') for x in PROTO_SRCS]
+
+genrule(
+    name='generate-proto-sources',
+    srcs=PROTO_SRCS,
+    cmd='mkdir -p $OUT && pwd && protoc --proto_path=. --cpp_out=$OUT *.proto',
+    out='output', )
+
+for header_filename in HEADER_FILENAMES:
+    genrule(name=header_filename,
+            cmd='mkdir -p `dirname $OUT` '
+            ' && cp $(location :generate-proto-sources)/{} $OUT'.format(
+                header_filename),
+            out=header_filename, )
+for cc_filename in CC_FILENAMES:
+    genrule(
+        name=cc_filename,
+        cmd='mkdir -p `dirname $OUT` '
+        ' && cp $(location :generate-proto-sources)/*.cc `dirname $OUT` '
+        ' && cp $(location :generate-proto-sources)/*.h `dirname $OUT`'.format(
+            cc_filename),
+        out=cc_filename, )
+
+cxx_library(name='if',
+            header_namespace="hbase/if",
+            exported_headers=[':' + x for x in HEADER_FILENAMES],
+            srcs=[':' + x for x in CC_FILENAMES],
+            deps=['//third-party:protobuf'],
+            visibility=['PUBLIC', ],
+            exported_deps=['//third-party:protobuf'])

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Cell.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Cell.proto b/hbase-native-client/src/hbase/if/Cell.proto
new file mode 100644
index 0000000..2c61035
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Cell.proto
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+
+// Cell and KeyValue protos
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "CellProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+/**
+ * The type of the key in a Cell
+ */
+enum CellType {
+    MINIMUM = 0;
+    PUT = 4;
+
+    DELETE = 8;
+    DELETE_COLUMN = 12;
+    DELETE_FAMILY = 14;
+
+    // MAXIMUM is used when searching; you look from maximum on down.
+    MAXIMUM = 255;
+}
+
+/**
+ * Protocol buffer version of Cell.
+ */
+message Cell {
+  optional bytes row = 1;
+  optional bytes family = 2;
+  optional bytes qualifier = 3;
+  optional uint64 timestamp = 4;
+  optional CellType cell_type = 5;
+  optional bytes value = 6;
+  optional bytes tags = 7;
+}
+
+/**
+ * Protocol buffer version of KeyValue.
+ * It doesn't have those transient parameters
+ */
+message KeyValue {
+  required bytes row = 1;
+  required bytes family = 2;
+  required bytes qualifier = 3;
+  optional uint64 timestamp = 4;
+  optional CellType key_type = 5;
+  optional bytes value = 6;
+  optional bytes tags = 7;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Client.proto b/hbase-native-client/src/hbase/if/Client.proto
new file mode 100644
index 0000000..8a4d459
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Client.proto
@@ -0,0 +1,478 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for Client service.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ClientProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Filter.proto";
+import "Cell.proto";
+import "Comparator.proto";
+import "MapReduce.proto";
+
+/**
+ * The protocol buffer version of Authorizations.
+ */
+message Authorizations {
+  repeated string label = 1;
+}
+
+/**
+ * The protocol buffer version of CellVisibility.
+ */
+message CellVisibility {
+  required string expression = 1;
+}
+
+/**
+ * Container for a list of column qualifier names of a family.
+ */
+message Column {
+  required bytes family = 1;
+  repeated bytes qualifier = 2;
+}
+
+/**
+ * Consistency defines the expected consistency level for an operation.
+ */
+enum Consistency {
+  STRONG   = 0;
+  TIMELINE = 1;
+}
+
+/**
+ * The protocol buffer version of Get.
+ * Unless existence_only is specified, return all the requested data
+ * for the row that matches exactly.
+ */
+message Get {
+  required bytes row = 1;
+  repeated Column column = 2;
+  repeated NameBytesPair attribute = 3;
+  optional Filter filter = 4;
+  optional TimeRange time_range = 5;
+  optional uint32 max_versions = 6 [default = 1];
+  optional bool cache_blocks = 7 [default = true];
+  optional uint32 store_limit = 8;
+  optional uint32 store_offset = 9;
+
+  // The result isn't asked for, just check for
+  // the existence.
+  optional bool existence_only = 10 [default = false];
+
+  optional Consistency consistency = 12 [default = STRONG];
+  repeated ColumnFamilyTimeRange cf_time_range = 13;
+}
+
+message Result {
+  // Result includes the Cells or else it just has a count of Cells
+  // that are carried otherwise.
+  repeated Cell cell = 1;
+  // The below count is set when the associated cells are
+  // not part of this protobuf message; they are passed alongside
+  // and then this Message is just a placeholder with metadata.
+  // The count is needed to know how many to peel off the block of Cells as
+  // ours.  NOTE: This is different from the pb managed cell_count of the
+  // 'cell' field above which is non-null when the cells are pb'd.
+  optional int32 associated_cell_count = 2;
+
+  // used for Get to check existence only. Not set if existence_only was not set to true
+  //  in the query.
+  optional bool exists = 3;
+
+  // Whether or not the results are coming from possibly stale data 
+  optional bool stale = 4 [default = false];
+
+  // Whether or not the entire result could be returned. Results will be split when
+  // the RPC chunk size limit is reached. Partial results contain only a subset of the
+  // cells for a row and must be combined with a result containing the remaining cells
+  // to form a complete result
+  optional bool partial = 5 [default = false];
+}
+
+/**
+ * The get request. Perform a single Get operation.
+ */
+message GetRequest {
+  required RegionSpecifier region = 1;
+  required Get get = 2;
+}
+
+message GetResponse {
+  optional Result result = 1;
+}
+
+/**
+ * Condition to check if the value of a given cell (row,
+ * family, qualifier) matches a value via a given comparator.
+ *
+ * Condition is used in check and mutate operations.
+ */
+message Condition {
+  required bytes row = 1;
+  required bytes family = 2;
+  required bytes qualifier = 3;
+  required CompareType compare_type = 4;
+  required Comparator comparator = 5;
+}
+
+
+/**
+ * A specific mutation inside a mutate request.
+ * It can be an append, increment, put or delete based
+ * on the mutation type.  It can be fully filled in or
+ * only metadata present because data is being carried
+ * elsewhere outside of pb.
+ */
+message MutationProto {
+  optional bytes row = 1;
+  optional MutationType mutate_type = 2;
+  repeated ColumnValue column_value = 3;
+  optional uint64 timestamp = 4;
+  repeated NameBytesPair attribute = 5;
+  optional Durability durability = 6 [default = USE_DEFAULT];
+
+  // For some mutations, a result may be returned, in which case,
+  // time range can be specified for potential performance gain
+  optional TimeRange time_range = 7;
+  // The below count is set when the associated cells are NOT
+  // part of this protobuf message; they are passed alongside
+  // and then this Message is a placeholder with metadata.  The
+  // count is needed to know how many to peel off the block of Cells as
+  // ours.  NOTE: This is different from the pb managed cell_count of the
+  // 'cell' field above which is non-null when the cells are pb'd.
+  optional int32 associated_cell_count = 8;
+
+  optional uint64 nonce = 9;
+
+  enum Durability {
+    USE_DEFAULT  = 0;
+    SKIP_WAL     = 1;
+    ASYNC_WAL    = 2;
+    SYNC_WAL     = 3;
+    FSYNC_WAL    = 4;
+  }
+
+  enum MutationType {
+    APPEND = 0;
+    INCREMENT = 1;
+    PUT = 2;
+    DELETE = 3;
+  }
+
+  enum DeleteType {
+    DELETE_ONE_VERSION = 0;
+    DELETE_MULTIPLE_VERSIONS = 1;
+    DELETE_FAMILY = 2;
+    DELETE_FAMILY_VERSION = 3;
+  }
+
+  message ColumnValue {
+    required bytes family = 1;
+    repeated QualifierValue qualifier_value = 2;
+
+    message QualifierValue {
+      optional bytes qualifier = 1;
+      optional bytes value = 2;
+      optional uint64 timestamp = 3;
+      optional DeleteType delete_type = 4;
+      optional bytes tags = 5;
+    }
+  }
+}
+
+/**
+ * The mutate request. Perform a single Mutate operation.
+ *
+ * Optionally, you can specify a condition. The mutate
+ * will take place only if the condition is met.  Otherwise,
+ * the mutate will be ignored.  In the response result,
+ * parameter processed is used to indicate if the mutate
+ * actually happened.
+ */
+message MutateRequest {
+  required RegionSpecifier region = 1;
+  required MutationProto mutation = 2;
+  optional Condition condition = 3;
+  optional uint64 nonce_group = 4;
+}
+
+message MutateResponse {
+  optional Result result = 1;
+
+  // used for mutate to indicate processed only
+  optional bool processed = 2;
+}
+
+/**
+ * Instead of get from a table, you can scan it with optional filters.
+ * You can specify the row key range, time range, the columns/families
+ * to scan and so on.
+ *
+ * This scan is used the first time in a scan request. The response of
+ * the initial scan will return a scanner id, which should be used to
+ * fetch result batches later on before it is closed.
+ */
+message Scan {
+  repeated Column column = 1;
+  repeated NameBytesPair attribute = 2;
+  optional bytes start_row = 3;
+  optional bytes stop_row = 4;
+  optional Filter filter = 5;
+  optional TimeRange time_range = 6;
+  optional uint32 max_versions = 7 [default = 1];
+  optional bool cache_blocks = 8 [default = true];
+  optional uint32 batch_size = 9;
+  optional uint64 max_result_size = 10;
+  optional uint32 store_limit = 11;
+  optional uint32 store_offset = 12;
+  optional bool load_column_families_on_demand = 13; /* DO NOT add defaults to load_column_families_on_demand. */
+  optional bool small = 14;
+  optional bool reversed = 15 [default = false];
+  optional Consistency consistency = 16 [default = STRONG];
+  optional uint32 caching = 17;
+  optional bool allow_partial_results = 18;
+  repeated ColumnFamilyTimeRange cf_time_range = 19;
+}
+
+/**
+ * A scan request. Initially, it should specify a scan. Later on, you
+ * can use the scanner id returned to fetch result batches with a different
+ * scan request.
+ *
+ * The scanner will remain open if there are more results, and it's not
+ * asked to be closed explicitly.
+ *
+ * You can fetch the results and ask the scanner to be closed to save
+ * a trip if you are not interested in remaining results.
+ */
+message ScanRequest {
+  optional RegionSpecifier region = 1;
+  optional Scan scan = 2;
+  optional uint64 scanner_id = 3;
+  optional uint32 number_of_rows = 4;
+  optional bool close_scanner = 5;
+  optional uint64 next_call_seq = 6;
+  optional bool client_handles_partials = 7;
+  optional bool client_handles_heartbeats = 8;
+  optional bool track_scan_metrics = 9;
+  optional bool renew = 10 [default = false];
+}
+
+/**
+ * The scan response. If there are no more results, more_results will
+ * be false.  If it is not specified, it means there are more.
+ */
+message ScanResponse {
+  // This field is filled in if we are doing cellblocks.  A cellblock is made up
+  // of all Cells serialized out as one cellblock BUT responses from a server
+  // have their Cells grouped by Result.  So we can reconstitute the
+  // Results on the client-side, this field is a list of counts of Cells
+  // in each Result that makes up the response.  For example, if this field
+  // has 3, 3, 3 in it, then we know that on the client, we are to make
+  // three Results each of three Cells each.
+  repeated uint32 cells_per_result = 1;
+
+  optional uint64 scanner_id = 2;
+  optional bool more_results = 3;
+  optional uint32 ttl = 4;
+  // If cells are not carried in an accompanying cellblock, then they are pb'd here.
+  // This field is mutually exclusive with cells_per_result (since the Cells will
+  // be inside the pb'd Result)
+  repeated Result results = 5;
+  optional bool stale = 6;
+
+  // This field is filled in if we are doing cellblocks. In the event that a row
+  // could not fit all of its cells into a single RPC chunk, the results will be
+  // returned as partials, and reconstructed into a complete result on the client
+  // side. This field is a list of flags indicating whether or not the result
+  // that the cells belong to is a partial result. For example, if this field
+  // has false, false, true in it, then we know that on the client side, we need to
+  // make another RPC request since the last result was only a partial.
+  repeated bool partial_flag_per_result = 7;
+
+  // A server may choose to limit the number of results returned to the client for
+  // reasons such as the size in bytes or quantity of results accumulated. This field
+  // will true when more results exist in the current region.
+  optional bool more_results_in_region = 8;
+  
+  // This field is filled in if the server is sending back a heartbeat message.
+  // Heartbeat messages are sent back to the client to prevent the scanner from
+  // timing out. Seeing a heartbeat message communicates to the Client that the
+  // server would have continued to scan had the time limit not been reached.
+  optional bool heartbeat_message = 9;
+  
+  // This field is filled in if the client has requested that scan metrics be tracked.
+  // The metrics tracked here are sent back to the client to be tracked together with 
+  // the existing client side metrics.
+  optional ScanMetrics scan_metrics = 10;
+}
+
+/**
+ * Atomically bulk load multiple HFiles (say from different column families)
+ * into an open region.
+ */
+message BulkLoadHFileRequest {
+  required RegionSpecifier region = 1;
+  repeated FamilyPath family_path = 2;
+  optional bool assign_seq_num = 3;
+
+  message FamilyPath {
+    required bytes family = 1;
+    required string path = 2;
+  }
+}
+
+message BulkLoadHFileResponse {
+  required bool loaded = 1;
+}
+
+message CoprocessorServiceCall {
+  required bytes row = 1;
+  required string service_name = 2;
+  required string method_name = 3;
+  required bytes request = 4;
+}
+
+message CoprocessorServiceResult {
+  optional NameBytesPair value = 1;
+}
+
+message CoprocessorServiceRequest {
+  required RegionSpecifier region = 1;
+  required CoprocessorServiceCall call = 2;
+}
+
+message CoprocessorServiceResponse {
+  required RegionSpecifier region = 1;
+  required NameBytesPair value = 2;
+}
+
+// Either a Get or a Mutation
+message Action {
+  // If part of a multi action, useful aligning
+  // result with what was originally submitted.
+  optional uint32 index = 1;
+  optional MutationProto mutation = 2;
+  optional Get get = 3;
+  optional CoprocessorServiceCall service_call = 4;
+}
+
+/**
+ * Actions to run against a Region.
+ */
+message RegionAction {
+  required RegionSpecifier region = 1;
+  // When set, run mutations as atomic unit.
+  optional bool atomic = 2;
+  repeated Action action = 3;
+}
+
+/*
+* Statistics about the current load on the region
+*/
+message RegionLoadStats {
+  // Percent load on the memstore. Guaranteed to be positive, between 0 and 100.
+  optional int32 memstoreLoad = 1 [default = 0];
+  // Percent JVM heap occupancy. Guaranteed to be positive, between 0 and 100.
+  // We can move this to "ServerLoadStats" should we develop them.
+  optional int32 heapOccupancy = 2 [default = 0];
+  // Compaction pressure. Guaranteed to be positive, between 0 and 100.
+  optional int32 compactionPressure = 3 [default = 0];
+}
+
+message MultiRegionLoadStats{
+  repeated RegionSpecifier region = 1;
+  repeated RegionLoadStats stat = 2;
+}
+
+/**
+ * Either a Result or an Exception NameBytesPair (keyed by
+ * exception name whose value is the exception stringified)
+ * or maybe empty if no result and no exception.
+ */
+message ResultOrException {
+  // If part of a multi call, save original index of the list of all
+  // passed so can align this response w/ original request.
+  optional uint32 index = 1;
+  optional Result result = 2;
+  optional NameBytesPair exception = 3;
+  // result if this was a coprocessor service call
+  optional CoprocessorServiceResult service_result = 4;
+  // current load on the region
+  optional RegionLoadStats loadStats = 5 [deprecated=true];
+}
+
+/**
+ * The result of a RegionAction.
+ */
+message RegionActionResult {
+  repeated ResultOrException resultOrException = 1;
+  // If the operation failed globally for this region, this exception is set
+  optional NameBytesPair exception = 2;
+}
+
+/**
+ * Execute a list of actions on a given region in order.
+ * Nothing prevents a request to contains a set of RegionAction on the same region.
+ * For this reason, the matching between the MultiRequest and the MultiResponse is not
+ *  done by the region specifier but by keeping the order of the RegionActionResult vs.
+ *  the order of the RegionAction.
+ */
+message MultiRequest {
+  repeated RegionAction regionAction = 1;
+  optional uint64 nonceGroup = 2;
+  optional Condition condition = 3;
+}
+
+message MultiResponse {
+  repeated RegionActionResult regionActionResult = 1;
+  // used for mutate to indicate processed only
+  optional bool processed = 2;
+  optional MultiRegionLoadStats regionStatistics = 3;
+}
+
+
+service ClientService {
+  rpc Get(GetRequest)
+    returns(GetResponse);
+
+  rpc Mutate(MutateRequest)
+    returns(MutateResponse);
+
+  rpc Scan(ScanRequest)
+    returns(ScanResponse);
+
+  rpc BulkLoadHFile(BulkLoadHFileRequest)
+    returns(BulkLoadHFileResponse);
+
+  rpc ExecService(CoprocessorServiceRequest)
+    returns(CoprocessorServiceResponse);
+    
+  rpc ExecRegionServerService(CoprocessorServiceRequest)
+    returns(CoprocessorServiceResponse);
+
+  rpc Multi(MultiRequest)
+    returns(MultiResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/ClusterId.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/ClusterId.proto b/hbase-native-client/src/hbase/if/ClusterId.proto
new file mode 100644
index 0000000..aed8cfc
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/ClusterId.proto
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are shared throughout HBase
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ClusterIdProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+/**
+ * Content of the '/hbase/hbaseid', cluster id, znode.
+ * Also cluster of the ${HBASE_ROOTDIR}/hbase.id file.
+ */
+message ClusterId {
+  // This is the cluster id, a uuid as a String
+  required string cluster_id = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/ClusterStatus.proto b/hbase-native-client/src/hbase/if/ClusterStatus.proto
new file mode 100644
index 0000000..54bc0c3
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/ClusterStatus.proto
@@ -0,0 +1,227 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for ClustStatus
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ClusterStatusProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "ClusterId.proto";
+import "FS.proto";
+
+message RegionState {
+  required RegionInfo region_info = 1;
+  required State state = 2;
+  optional uint64 stamp = 3;
+  enum State {
+    OFFLINE = 0;       // region is in an offline state
+    PENDING_OPEN = 1;  // sent rpc to server to open but has not begun
+    OPENING = 2;       // server has begun to open but not yet done
+    OPEN = 3;          // server opened region and updated meta
+    PENDING_CLOSE = 4; // sent rpc to server to close but has not begun
+    CLOSING = 5;       // server has begun to close but not yet done
+    CLOSED = 6;        // server closed region and updated meta
+    SPLITTING = 7;     // server started split of a region
+    SPLIT = 8;         // server completed split of a region
+    FAILED_OPEN = 9;   // failed to open, and won't retry any more
+    FAILED_CLOSE = 10; // failed to close, and won't retry any more
+    MERGING = 11;      // server started merge a region
+    MERGED = 12;       // server completed merge of a region
+    SPLITTING_NEW = 13;  // new region to be created when RS splits a parent
+                       // region but hasn't be created yet, or master doesn't
+                       // know it's already created
+    MERGING_NEW = 14;  // new region to be created when RS merges two
+                       // daughter regions but hasn't be created yet, or
+                       // master doesn't know it's already created
+  }
+}
+
+message RegionInTransition {
+  required RegionSpecifier spec = 1;
+  required RegionState region_state = 2;
+}
+
+/**
+ * sequence Id of a store
+ */
+message StoreSequenceId {
+  required bytes family_name = 1;
+  required uint64 sequence_id = 2;
+}
+
+/**
+ * contains a sequence id of a region which should be the minimum of its store sequence ids and
+ * list of sequence ids of the region's stores
+ */
+message RegionStoreSequenceIds {
+  required uint64 last_flushed_sequence_id = 1;
+  repeated StoreSequenceId store_sequence_id = 2;
+}
+
+message RegionLoad {
+  /** the region specifier */
+  required RegionSpecifier region_specifier = 1;
+
+  /** the number of stores for the region */
+  optional uint32 stores = 2;
+
+  /** the number of storefiles for the region */
+  optional uint32 storefiles = 3;
+
+  /** the total size of the store files for the region, uncompressed, in MB */
+  optional uint32 store_uncompressed_size_MB = 4;
+
+  /** the current total size of the store files for the region, in MB */
+  optional uint32 storefile_size_MB = 5;
+
+  /** the current size of the memstore for the region, in MB */
+  optional uint32 memstore_size_MB = 6;
+
+  /**
+   * The current total size of root-level store file indexes for the region,
+   * in MB. The same as {@link #rootIndexSizeKB} but in MB.
+   */
+  optional uint32 storefile_index_size_MB = 7;
+
+  /** the current total read requests made to region */
+  optional uint64 read_requests_count = 8;
+
+  /** the current total write requests made to region */
+  optional uint64 write_requests_count = 9;
+
+  /** the total compacting key values in currently running compaction */
+  optional uint64 total_compacting_KVs = 10;
+
+  /** the completed count of key values in currently running compaction */
+  optional uint64 current_compacted_KVs = 11;
+
+   /** The current total size of root-level indexes for the region, in KB. */
+  optional uint32 root_index_size_KB = 12;
+
+  /** The total size of all index blocks, not just the root level, in KB. */
+  optional uint32 total_static_index_size_KB = 13;
+
+  /**
+   * The total size of all Bloom filter blocks, not just loaded into the
+   * block cache, in KB.
+   */
+  optional uint32 total_static_bloom_size_KB = 14;
+
+  /** the most recent sequence Id from cache flush */
+  optional uint64 complete_sequence_id = 15;
+
+  /** The current data locality for region in the regionserver */
+  optional float data_locality = 16;
+
+  optional uint64 last_major_compaction_ts = 17 [default = 0];
+
+  /** the most recent sequence Id of store from cache flush */
+  repeated StoreSequenceId store_complete_sequence_id = 18;
+
+  /** the current total filtered read requests made to region */
+  optional uint64 filtered_read_requests_count = 19;
+}
+
+/* Server-level protobufs */
+
+message ReplicationLoadSink {
+  required uint64 ageOfLastAppliedOp = 1;
+  required uint64 timeStampsOfLastAppliedOp = 2;
+}
+
+message ReplicationLoadSource {
+  required string peerID = 1;
+  required uint64 ageOfLastShippedOp = 2;
+  required uint32 sizeOfLogQueue = 3;
+  required uint64 timeStampOfLastShippedOp = 4;
+  required uint64 replicationLag = 5;
+}
+
+message ServerLoad {
+  /** Number of requests since last report. */
+  optional uint64 number_of_requests = 1;
+
+  /** Total Number of requests from the start of the region server. */
+  optional uint64 total_number_of_requests = 2;
+
+  /** the amount of used heap, in MB. */
+  optional uint32 used_heap_MB = 3;
+
+  /** the maximum allowable size of the heap, in MB. */
+  optional uint32 max_heap_MB = 4;
+
+  /** Information on the load of individual regions. */
+  repeated RegionLoad region_loads = 5;
+
+  /**
+   * Regionserver-level coprocessors, e.g., WALObserver implementations.
+   * Region-level coprocessors, on the other hand, are stored inside RegionLoad
+   * objects.
+   */
+  repeated Coprocessor coprocessors = 6;
+
+  /**
+   * Time when incremental (non-total) counts began being calculated (e.g. number_of_requests)
+   * time is measured as the difference, measured in milliseconds, between the current time
+   * and midnight, January 1, 1970 UTC.
+   */
+  optional uint64 report_start_time = 7;
+
+  /**
+   * Time when report was generated.
+   * time is measured as the difference, measured in milliseconds, between the current time
+   * and midnight, January 1, 1970 UTC.
+   */
+  optional uint64 report_end_time = 8;
+
+  /**
+   * The port number that this region server is hosing an info server on.
+   */
+  optional uint32 info_server_port = 9;
+
+  /**
+   * The replicationLoadSource for the replication Source status of this region server.
+   */
+  repeated ReplicationLoadSource replLoadSource = 10;
+
+  /**
+   * The replicationLoadSink for the replication Sink status of this region server.
+   */
+  optional ReplicationLoadSink replLoadSink = 11;
+}
+
+message LiveServerInfo {
+  required ServerName server = 1;
+  required ServerLoad server_load = 2;
+}
+
+message ClusterStatus {
+  optional HBaseVersionFileContent hbase_version = 1;
+  repeated LiveServerInfo live_servers = 2;
+  repeated ServerName dead_servers = 3;
+  repeated RegionInTransition regions_in_transition = 4;
+  optional ClusterId cluster_id = 5;
+  repeated Coprocessor master_coprocessors = 6;
+  optional ServerName master = 7;
+  repeated ServerName backup_masters = 8;
+  optional bool balancer_on = 9;
+}


[19/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/load-client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/load-client.cc b/hbase-native-client/core/load-client.cc
deleted file mode 100644
index 8cceeef..0000000
--- a/hbase-native-client/core/load-client.cc
+++ /dev/null
@@ -1,390 +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 <folly/Logging.h>
-#include <folly/Random.h>
-#include <gflags/gflags.h>
-
-#include <atomic>
-#include <chrono>
-#include <iostream>
-#include <thread>
-
-#include "core/client.h"
-#include "core/get.h"
-#include "core/put.h"
-#include "core/table.h"
-#include "serde/table-name.h"
-#include "utils/time-util.h"
-
-using hbase::Client;
-using hbase::Configuration;
-using hbase::Get;
-using hbase::Put;
-using hbase::Table;
-using hbase::pb::TableName;
-using hbase::TimeUtil;
-using folly::Random;
-
-DEFINE_string(table, "load_test_table", "What table to do the reads and writes with");
-DEFINE_string(families, "f", "comma separated list of column family names");
-DEFINE_string(conf, "", "Conf directory to read the config from (optional)");
-DEFINE_string(zookeeper, "localhost:2181", "What zk quorum to talk to");
-DEFINE_string(znode, "/hbase", "parent znode");
-DEFINE_uint64(num_rows, 1'000'000, "How many rows to write and read");
-DEFINE_uint64(num_cols, 1000, "How many columns there are in a row");
-DEFINE_int32(threads, 10, "How many client threads");
-DEFINE_int32(batch_num_rows, 100, "number of rows in one multi-get / multi-put");
-DEFINE_uint64(report_num_rows, 5000, "How frequent we should report the progress");
-DEFINE_bool(gets, true, "perform gets");
-DEFINE_bool(scans, true, "perform scans");
-DEFINE_bool(puts, true, "perform put's");
-DEFINE_bool(appends, true, "perform append's");
-
-static constexpr const char *kNumColumn = "num";
-static constexpr const char *incrPrefix = "i";
-static constexpr const char *appendPrefix = "a";
-
-std::string PrefixZero(int total_width, int num) {
-  std::string str = std::to_string(num);
-  int prefix_len = total_width - str.length();
-  if (prefix_len > 0) {
-    return std::string(prefix_len, '0') + str;
-  }
-  return str;
-}
-
-bool Verify(std::shared_ptr<hbase::Result> result, std::string family, int m) {
-  auto col = std::to_string(m);
-  if (!result->Value(family, col)) {
-    LOG(ERROR) << "Column:" << col << " is not found for " << result->Row();
-    return false;
-  }
-  auto l = *(result->Value(family, col));
-  if (l != col) {
-    LOG(ERROR) << "value " << *(result->Value(family, "1")) << " is not " << col;
-    return false;
-  }
-  if (FLAGS_appends) {
-    if (!result->Value(family, incrPrefix + col)) {
-      LOG(ERROR) << "Column:" << (incrPrefix + col) << " is not found for " << result->Row();
-      return false;
-    }
-    auto int_val = hbase::BytesUtil::ToInt64(*(result->Value(family, incrPrefix + col)));
-    if (int_val != m) {
-      LOG(ERROR) << "value is not " << col << " for " << result->Row();
-      return false;
-    }
-    if (!result->Value(family, appendPrefix + col)) {
-      LOG(ERROR) << "Column:" << (appendPrefix + col) << " is not found for " << result->Row();
-      return false;
-    }
-    l = *(result->Value(family, appendPrefix + col));
-    if (l != col) {
-      LOG(ERROR) << "value " << *(result->Value(family, "1")) << " is not " << col;
-      return false;
-    }
-  }
-
-  return true;
-}
-
-bool Verify(std::shared_ptr<hbase::Result> result, const std::string &row,
-            const std::vector<std::string> &families) {
-  if (result == nullptr || result->IsEmpty()) {
-    LOG(ERROR) << "didn't get result";
-    return false;
-  }
-  if (result->Row().compare(row) != 0) {
-    LOG(ERROR) << "row " << result->Row() << " is not the expected: " << row;
-    return false;
-  }
-  // Test the values
-  for (auto family : families) {
-    if (!result->Value(family, kNumColumn)) {
-      LOG(ERROR) << "Column:" << kNumColumn << " is not found for " << result->Row();
-      return false;
-    }
-    auto cols = std::stoi(*(result->Value(family, kNumColumn)));
-    VLOG(3) << "Result for row:" << row << " contains " << std::to_string(cols) << " columns";
-    for (int m = 1; m <= cols; m++) {
-      if (!Verify(result, family, m)) return false;
-    }
-  }
-  return true;
-}
-
-bool DoScan(int iteration, uint64_t max_row, uint64_t rows, std::unique_ptr<Table> table,
-            const std::vector<std::string> &families) {
-  hbase::Scan scan{};
-  auto start = iteration * rows;
-  auto end = start + rows;
-  auto width = std::to_string(max_row).length();
-  scan.SetStartRow(PrefixZero(width, start));
-  if (end != max_row && end != max_row + 1) {
-    scan.SetStopRow(PrefixZero(width, end));
-  }
-
-  auto start_ns = TimeUtil::GetNowNanos();
-  auto scanner = table->Scan(scan);
-
-  auto cnt = 0;
-  auto r = scanner->Next();
-  while (r != nullptr) {
-    auto row = PrefixZero(width, start + cnt);
-    if (!Verify(r, row, families)) {
-      return false;
-    }
-    cnt++;
-    r = scanner->Next();
-    if (cnt != 0 && cnt % FLAGS_report_num_rows == 0) {
-      LOG(INFO) << "(Thread " << iteration << ") "
-                << "Scan iterated over " << cnt << " results in "
-                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-    }
-  }
-  if (cnt != rows) {
-    LOG(ERROR) << "(Thread " << iteration << ") "
-               << "Expected number of results does not match. expected:" << rows
-               << ", actual:" << cnt;
-    return false;
-  }
-  LOG(INFO) << "(Thread " << iteration << ") "
-            << "scanned " << std::to_string(cnt) << " rows in " << TimeUtil::ElapsedMillis(start_ns)
-            << " ms.";
-  return true;
-}
-
-bool DoGet(int iteration, uint64_t max_row, uint64_t rows, std::unique_ptr<Table> table,
-           const std::vector<std::string> &families, uint64_t batch_num_rows) {
-  auto width = std::to_string(max_row).length();
-  auto start_ns = TimeUtil::GetNowNanos();
-  for (uint64_t k = iteration; k <= max_row;) {
-    uint64_t total_read = 0;
-    std::vector<hbase::Get> gets;
-    for (uint64_t i = 0; i < batch_num_rows && k <= max_row; ++i, k += FLAGS_threads) {
-      std::string row = PrefixZero(width, k);
-      hbase::Get get(row);
-      gets.push_back(get);
-    }
-    VLOG(3) << "getting for " << batch_num_rows << " rows";
-    auto results = table->Get(gets);
-    if (results.size() != gets.size()) {
-      LOG(ERROR) << "(Thread " << iteration << ") "
-                 << "Expected number of results does not match. expected:" << gets.size()
-                 << ", actual:" << results.size();
-      return false;
-    }
-    for (uint64_t i = 0; i < batch_num_rows && i < results.size(); ++i) {
-      if (!Verify(results[i], gets[i].row(), families)) {
-        return false;
-      }
-    }
-    total_read += gets.size();
-    if (total_read != 0 && total_read % FLAGS_report_num_rows == 0) {
-      LOG(INFO) << "(Thread " << iteration << ") "
-                << "Sent  " << total_read << " Multi-Get requests in "
-                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-    }
-    k += batch_num_rows;
-  }
-  LOG(INFO) << "(Thread " << iteration << ") "
-            << "Sent " << rows << " gets"
-            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  return true;
-}
-
-void DoPut(int iteration, uint64_t max_row, uint64_t rows, int cols, std::unique_ptr<Table> table,
-           const std::vector<std::string> &families) {
-  auto start_ns = TimeUtil::GetNowNanos();
-  auto width = std::to_string(max_row).length();
-  for (uint64_t j = 0; j < rows; j++) {
-    std::string row = PrefixZero(width, iteration * rows + j);
-    auto put = Put{row};
-    for (auto family : families) {
-      auto n_cols = Random::rand32(1, cols);
-      put.AddColumn(family, kNumColumn, std::to_string(n_cols));
-      for (unsigned int k = 1; k <= n_cols; k++) {
-        put.AddColumn(family, std::to_string(k), std::to_string(k));
-      }
-    }
-    table->Put(put);
-    if ((j + 1) % FLAGS_report_num_rows == 0) {
-      LOG(INFO) << "(Thread " << iteration << ") "
-                << "Written " << std::to_string(j + 1) << " rows in "
-                << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-    }
-  }
-  LOG(INFO) << "(Thread " << iteration << ") "
-            << "written " << std::to_string(rows) << " rows"
-            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-}
-
-bool DoAppendIncrement(int iteration, uint64_t max_row, uint64_t rows, int cols,
-                       std::unique_ptr<Table> table, const std::vector<std::string> &families) {
-  auto start_ns = TimeUtil::GetNowNanos();
-  auto width = std::to_string(max_row).length();
-  for (uint64_t j = 0; j < rows; j++) {
-    std::string row = PrefixZero(width, iteration * rows + j);
-    hbase::Get get(row);
-    auto result = table->Get(get);
-    for (auto family : families) {
-      auto n_cols = std::stoi(*(result->Value(family, kNumColumn)));
-      for (unsigned int k = 1; k <= n_cols; k++) {
-        table->Increment(
-            hbase::Increment{row}.AddColumn(family, incrPrefix + std::to_string(k), k));
-        if (!table->Append(hbase::Append{row}.Add(family, appendPrefix + std::to_string(k),
-                                                  std::to_string(k)))) {
-          LOG(ERROR) << "(Thread " << iteration << ") "
-                     << "append for " << row << " family: " << family << " failed";
-          return false;
-        }
-      }
-    }
-    if ((j + 1) % FLAGS_report_num_rows == 0)
-      LOG(INFO) << "(Thread " << iteration << ") "
-                << "Written " << std::to_string(j + 1) << " increments"
-                << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-  LOG(INFO) << "(Thread " << iteration << ") "
-            << "written " << std::to_string(rows) << " increments"
-            << " in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  return true;
-}
-
-int main(int argc, char *argv[]) {
-  gflags::SetUsageMessage("Load client to manipulate multiple rows from HBase on the comamnd line");
-  gflags::ParseCommandLineFlags(&argc, &argv, true);
-  google::InitGoogleLogging(argv[0]);
-  google::InstallFailureSignalHandler();
-  FLAGS_logtostderr = 1;
-  FLAGS_stderrthreshold = 1;
-
-  if (FLAGS_batch_num_rows < 1) {
-    LOG(ERROR) << "size of multi get should be positive";
-    return -1;
-  }
-  if (!FLAGS_gets && !FLAGS_scans && !FLAGS_puts) {
-    LOG(ERROR) << "Must perform at least Get or Put operations";
-    return -1;
-  }
-  std::shared_ptr<Configuration> conf = nullptr;
-  if (FLAGS_conf == "") {
-    // Configuration
-    conf = std::make_shared<Configuration>();
-    conf->Set("hbase.zookeeper.quorum", FLAGS_zookeeper);
-    conf->Set("zookeeper.znode.parent", FLAGS_znode);
-  } else {
-    setenv("HBASE_CONF", FLAGS_conf.c_str(), 1);
-    hbase::HBaseConfigurationLoader loader;
-    conf = std::make_shared<Configuration>(loader.LoadDefaultResources().value());
-  }
-  auto tn = std::make_shared<TableName>(folly::to<TableName>(FLAGS_table));
-  auto num_puts = FLAGS_num_rows;
-
-  auto client = std::make_unique<Client>(*conf);
-
-  // Do the Put requests
-
-  std::vector<std::string> families;
-  std::size_t pos = 0, found;
-  while ((found = FLAGS_families.find_first_of(',', pos)) != std::string::npos) {
-    families.push_back(FLAGS_families.substr(pos, found - pos));
-    pos = found + 1;
-  }
-  families.push_back(FLAGS_families.substr(pos));
-
-  int rows = FLAGS_num_rows / FLAGS_threads;
-  if (FLAGS_num_rows % FLAGS_threads != 0) rows++;
-  int cols = FLAGS_num_cols;
-  std::atomic<int8_t> succeeded{1};  // not using bool since we want atomic &=
-  if (FLAGS_puts) {
-    LOG(INFO) << "Sending put requests";
-    auto start_ns = TimeUtil::GetNowNanos();
-    std::vector<std::thread> writer_threads;
-    for (int i = 0; i < FLAGS_threads; i++) {
-      writer_threads.push_back(std::thread([&, i] {
-        auto table = client->Table(*tn);
-        DoPut(i, FLAGS_num_rows - 1, rows, cols, std::move(table), families);
-      }));
-    }
-    for (auto &t : writer_threads) {
-      t.join();
-    }
-    LOG(INFO) << "Successfully sent  " << num_puts << " Put requests in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-  if (FLAGS_appends) {
-    LOG(INFO) << "Sending append/increment requests";
-    auto start_ns = TimeUtil::GetNowNanos();
-    std::vector<std::thread> writer_threads;
-    for (int i = 0; i < FLAGS_threads; i++) {
-      writer_threads.push_back(std::thread([&, i] {
-        auto table = client->Table(*tn);
-        succeeded &=
-            DoAppendIncrement(i, FLAGS_num_rows - 1, rows, cols, std::move(table), families);
-      }));
-    }
-    for (auto &t : writer_threads) {
-      t.join();
-    }
-    LOG(INFO) << "Successfully sent  " << num_puts << " append requests in "
-              << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-
-  if (FLAGS_scans) {
-    LOG(INFO) << "Sending scan requests";
-    auto start_ns = TimeUtil::GetNowNanos();
-    std::vector<std::thread> reader_threads;
-    for (int i = 0; i < FLAGS_threads; i++) {
-      reader_threads.push_back(std::thread([&, i] {
-        auto table1 = client->Table(*tn);
-        succeeded &= DoScan(i, FLAGS_num_rows - 1, rows, std::move(table1), families);
-      }));
-    }
-    for (auto &t : reader_threads) {
-      t.join();
-    }
-
-    LOG(INFO) << (succeeded.load() ? "Successfully " : "Failed. ") << " scannned " << num_puts
-              << " rows in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-
-  if (FLAGS_gets) {
-    LOG(INFO) << "Sending get requests";
-    auto start_ns = TimeUtil::GetNowNanos();
-    std::vector<std::thread> reader_threads;
-    for (int i = 0; i < FLAGS_threads; i++) {
-      reader_threads.push_back(std::thread([&, i] {
-        auto table1 = client->Table(*tn);
-        succeeded &=
-            DoGet(i, FLAGS_num_rows - 1, rows, std::move(table1), families, FLAGS_batch_num_rows);
-      }));
-    }
-    for (auto &t : reader_threads) {
-      t.join();
-    }
-
-    LOG(INFO) << (succeeded.load() ? "Successful. " : "Failed. ") << " sent multi-get requests for "
-              << num_puts << " rows in " << TimeUtil::ElapsedMillis(start_ns) << " ms.";
-  }
-  client->Close();
-
-  return succeeded.load() ? 0 : -1;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/location-cache-retry-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache-retry-test.cc b/hbase-native-client/core/location-cache-retry-test.cc
deleted file mode 100644
index f154b69..0000000
--- a/hbase-native-client/core/location-cache-retry-test.cc
+++ /dev/null
@@ -1,112 +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 <gtest/gtest.h>
-
-#include "core/append.h"
-#include "core/cell.h"
-#include "core/client.h"
-#include "core/configuration.h"
-#include "core/delete.h"
-#include "core/get.h"
-#include "core/hbase-configuration-loader.h"
-#include "core/increment.h"
-#include "core/meta-utils.h"
-#include "core/put.h"
-#include "core/result.h"
-#include "core/table.h"
-#include "exceptions/exception.h"
-#include "serde/table-name.h"
-#include "test-util/test-util.h"
-#include "utils/bytes-util.h"
-
-using hbase::Cell;
-using hbase::Configuration;
-using hbase::Get;
-using hbase::MetaUtil;
-using hbase::RetriesExhaustedException;
-using hbase::Put;
-using hbase::Table;
-using hbase::TestUtil;
-
-using std::chrono_literals::operator"" s;
-
-class LocationCacheRetryTest : public ::testing::Test {
- public:
-  static std::unique_ptr<hbase::TestUtil> test_util;
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util = std::make_unique<hbase::TestUtil>();
-    test_util->StartMiniCluster(2);
-    test_util->conf()->SetInt("hbase.client.retries.number", 5);
-  }
-};
-
-std::unique_ptr<hbase::TestUtil> LocationCacheRetryTest::test_util = nullptr;
-
-TEST_F(LocationCacheRetryTest, GetFromMetaTable) {
-  auto tn = folly::to<hbase::pb::TableName>("hbase:meta");
-  auto row = "test1";
-
-  hbase::Client client(*LocationCacheRetryTest::test_util->conf());
-
-  // do a get against the other table, but not the actual table "t".
-  auto table = client.Table(tn);
-  hbase::Get get(row);
-  auto result = table->Get(get);
-
-  LocationCacheRetryTest::test_util->MoveRegion(MetaUtil::kMetaRegion, "");
-
-  std::this_thread::sleep_for(3s);  // sleep 3 sec
-
-  result = table->Get(get);
-}
-
-TEST_F(LocationCacheRetryTest, PutGet) {
-  LocationCacheRetryTest::test_util->CreateTable("t", "d");
-  LocationCacheRetryTest::test_util->CreateTable("t2", "d");
-
-  auto tn = folly::to<hbase::pb::TableName>("t");
-  auto tn2 = folly::to<hbase::pb::TableName>("t2");
-  auto row = "test1";
-
-  hbase::Client client(*LocationCacheRetryTest::test_util->conf());
-
-  // do a get against the other table, but not the actual table "t".
-  auto table = client.Table(tn);
-  auto table2 = client.Table(tn2);
-  hbase::Get get(row);
-  auto result = table2->Get(get);
-
-  // we should have already cached the location of meta right now. Now
-  // move the meta region to the other server so that we will get a NotServingRegionException
-  // when we do the actual location lookup request. If there is no invalidation
-  // of the meta's own location, then following put/get will result in retries exhausted.
-  LocationCacheRetryTest::test_util->MoveRegion(MetaUtil::kMetaRegion, "");
-
-  std::this_thread::sleep_for(3s);  // sleep 3 sec
-
-  table->Put(Put{row}.AddColumn("d", "1", "value1"));
-
-  result = table->Get(get);
-
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ("test1", result->Row());
-  EXPECT_EQ("value1", *(result->Value("d", "1")));
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/location-cache-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache-test.cc b/hbase-native-client/core/location-cache-test.cc
deleted file mode 100644
index fd96ff3..0000000
--- a/hbase-native-client/core/location-cache-test.cc
+++ /dev/null
@@ -1,164 +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/location-cache.h"
-
-#include <folly/Memory.h>
-#include <gtest/gtest.h>
-
-#include <chrono>
-
-#include "core/keyvalue-codec.h"
-#include "if/HBase.pb.h"
-#include "serde/table-name.h"
-#include "test-util/test-util.h"
-
-using hbase::Cell;
-using hbase::Configuration;
-using hbase::ConnectionPool;
-using hbase::MetaUtil;
-using hbase::LocationCache;
-using hbase::TestUtil;
-using hbase::KeyValueCodec;
-using std::chrono::milliseconds;
-
-class LocationCacheTest : public ::testing::Test {
- protected:
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util_ = std::make_unique<TestUtil>();
-    test_util_->StartMiniCluster(2);
-  }
-  static void TearDownTestCase() { test_util_.release(); }
-
-  virtual void SetUp() {}
-  virtual void TearDown() {}
-
- public:
-  static std::unique_ptr<TestUtil> test_util_;
-};
-
-std::unique_ptr<TestUtil> LocationCacheTest::test_util_ = nullptr;
-
-TEST_F(LocationCacheTest, TestGetMetaNodeContents) {
-  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
-  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
-  auto codec = std::make_shared<KeyValueCodec>();
-  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
-  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
-  auto f = cache.LocateMeta();
-  auto result = f.get();
-  ASSERT_FALSE(f.hasException());
-  ASSERT_TRUE(result.has_port());
-  ASSERT_TRUE(result.has_host_name());
-  cpu->stop();
-  io->stop();
-}
-
-TEST_F(LocationCacheTest, TestGetRegionLocation) {
-  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
-  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
-  auto codec = std::make_shared<KeyValueCodec>();
-  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
-  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
-
-  // If there is no table this should throw an exception
-  auto tn = folly::to<hbase::pb::TableName>("t");
-  auto row = "test";
-  ASSERT_ANY_THROW(cache.LocateFromMeta(tn, row).get(milliseconds(1000)));
-  LocationCacheTest::test_util_->CreateTable("t", "d");
-  auto loc = cache.LocateFromMeta(tn, row).get(milliseconds(1000));
-  ASSERT_TRUE(loc != nullptr);
-  cpu->stop();
-  io->stop();
-}
-
-TEST_F(LocationCacheTest, TestCaching) {
-  auto cpu = std::make_shared<wangle::CPUThreadPoolExecutor>(4);
-  auto io = std::make_shared<wangle::IOThreadPoolExecutor>(4);
-  auto codec = std::make_shared<KeyValueCodec>();
-  auto cp = std::make_shared<ConnectionPool>(io, cpu, codec, LocationCacheTest::test_util_->conf());
-  LocationCache cache{LocationCacheTest::test_util_->conf(), io, cpu, cp};
-
-  auto tn_1 = folly::to<hbase::pb::TableName>("t1");
-  auto tn_2 = folly::to<hbase::pb::TableName>("t2");
-  auto tn_3 = folly::to<hbase::pb::TableName>("t3");
-  auto row_a = "a";
-
-  // test location pulled from meta gets cached
-  ASSERT_ANY_THROW(cache.LocateRegion(tn_1, row_a).get(milliseconds(1000)));
-  ASSERT_ANY_THROW(cache.LocateFromMeta(tn_1, row_a).get(milliseconds(1000)));
-  LocationCacheTest::test_util_->CreateTable("t1", "d");
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_1, row_a));
-  auto loc = cache.LocateRegion(tn_1, row_a).get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_1, row_a));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_1, row_a));
-
-  // test with two regions
-  std::vector<std::string> keys;
-  keys.push_back("b");
-  LocationCacheTest::test_util_->CreateTable("t2", "d", keys);
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_2, "a"));
-  loc = cache.LocateRegion(tn_2, "a").get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_2, "a"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "a"));
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_2, "b"));
-  loc = cache.LocateRegion(tn_2, "b").get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_2, "b"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "b"));
-  ASSERT_TRUE(cache.IsLocationCached(tn_2, "ba"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_2, "ba"));
-
-  // test with three regions
-  keys.clear();
-  keys.push_back("b");
-  keys.push_back("c");
-  LocationCacheTest::test_util_->CreateTable("t3", "d", keys);
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_3, "c"));
-  ASSERT_FALSE(cache.IsLocationCached(tn_3, "ca"));
-  loc = cache.LocateRegion(tn_3, "ca").get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "c"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "c"));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "ca"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "ca"));
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_3, "b"));
-  loc = cache.LocateRegion(tn_3, "b").get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "b"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "b"));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "ba"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "ba"));
-
-  // clear second region
-  cache.ClearCachedLocation(tn_3, "b");
-  ASSERT_FALSE(cache.IsLocationCached(tn_3, "b"));
-
-  ASSERT_FALSE(cache.IsLocationCached(tn_3, "a"));
-  loc = cache.LocateRegion(tn_3, "a").get(milliseconds(1000));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "a"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "a"));
-  ASSERT_TRUE(cache.IsLocationCached(tn_3, "abc"));
-  ASSERT_EQ(loc, cache.GetCachedLocation(tn_3, "abc"));
-
-  cpu->stop();
-  io->stop();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/location-cache.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache.cc b/hbase-native-client/core/location-cache.cc
deleted file mode 100644
index b728d95..0000000
--- a/hbase-native-client/core/location-cache.cc
+++ /dev/null
@@ -1,334 +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/location-cache.h"
-
-#include <folly/Conv.h>
-#include <folly/Logging.h>
-#include <folly/io/IOBuf.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-
-#include <map>
-#include <shared_mutex>
-#include <utility>
-
-#include "connection/response.h"
-#include "connection/rpc-connection.h"
-#include "core/meta-utils.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "if/ZooKeeper.pb.h"
-#include "serde/region-info.h"
-#include "serde/server-name.h"
-#include "serde/zk.h"
-
-using hbase::pb::MetaRegionServer;
-using hbase::pb::ServerName;
-using hbase::pb::TableName;
-
-namespace hbase {
-
-LocationCache::LocationCache(std::shared_ptr<hbase::Configuration> conf,
-                             std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                             std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                             std::shared_ptr<ConnectionPool> cp)
-    : conf_(conf),
-      io_executor_(io_executor),
-      cpu_executor_(cpu_executor),
-      cp_(cp),
-      meta_promise_(nullptr),
-      meta_lock_(),
-      meta_util_(),
-      zk_(nullptr),
-      cached_locations_(),
-      locations_lock_() {
-  zk_quorum_ = ZKUtil::ParseZooKeeperQuorum(*conf_);
-  EnsureZooKeeperConnection();
-}
-
-LocationCache::~LocationCache() { CloseZooKeeperConnection(); }
-
-void LocationCache::CloseZooKeeperConnection() {
-  if (zk_ != nullptr) {
-    zookeeper_close(zk_);
-    zk_ = nullptr;
-    LOG(INFO) << "Closed connection to ZooKeeper.";
-  }
-}
-
-void LocationCache::EnsureZooKeeperConnection() {
-  if (zk_ == nullptr) {
-    LOG(INFO) << "Connecting to ZooKeeper. Quorum:" + zk_quorum_;
-    auto session_timeout = ZKUtil::SessionTimeout(*conf_);
-    zk_ = zookeeper_init(zk_quorum_.c_str(), nullptr, session_timeout, nullptr, nullptr, 0);
-  }
-}
-
-folly::Future<ServerName> LocationCache::LocateMeta() {
-  std::lock_guard<std::recursive_mutex> g(meta_lock_);
-  if (meta_promise_ == nullptr) {
-    this->RefreshMetaLocation();
-  }
-  return meta_promise_->getFuture().onError([&](const folly::exception_wrapper &ew) {
-    auto promise = InvalidateMeta();
-    if (promise) {
-      promise->setException(ew);
-    }
-    throw ew;
-    return ServerName{};
-  });
-}
-
-std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> LocationCache::InvalidateMeta() {
-  VLOG(2) << "Invalidating meta location";
-  std::lock_guard<std::recursive_mutex> g(meta_lock_);
-  if (meta_promise_ != nullptr) {
-    // return the unique_ptr back to the caller.
-    std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> ret = nullptr;
-    std::swap(ret, meta_promise_);
-    return ret;
-  } else {
-    return nullptr;
-  }
-}
-
-void LocationCache::RefreshMetaLocation() {
-  meta_promise_ = std::make_shared<folly::SharedPromise<ServerName>>();
-  auto p = meta_promise_;
-  cpu_executor_->add([this, p] {
-    std::lock_guard<std::recursive_mutex> g(meta_lock_);
-    p->setWith([&] { return this->ReadMetaLocation(); });
-  });
-}
-
-// Note: this is a blocking call to zookeeper
-ServerName LocationCache::ReadMetaLocation() {
-  auto buf = folly::IOBuf::create(4096);
-  ZkDeserializer derser;
-  EnsureZooKeeperConnection();
-
-  // This needs to be int rather than size_t as that's what ZK expects.
-  int len = buf->capacity();
-  std::string zk_node = ZKUtil::MetaZNode(*conf_);
-  int zk_result = zoo_get(this->zk_, zk_node.c_str(), 0,
-                          reinterpret_cast<char *>(buf->writableData()), &len, nullptr);
-  if (zk_result != ZOK || len < 9) {
-    LOG(ERROR) << "Error getting meta location.";
-    // We just close the zk connection, and let the upper levels retry.
-    CloseZooKeeperConnection();
-    throw std::runtime_error("Error getting meta location. Quorum: " + zk_quorum_);
-  }
-  buf->append(len);
-
-  MetaRegionServer mrs;
-  if (derser.Parse(buf.get(), &mrs) == false) {
-    LOG(ERROR) << "Unable to decode";
-    throw std::runtime_error("Error getting meta location (Unable to decode). Quorum: " +
-                             zk_quorum_);
-  }
-  return mrs.server();
-}
-
-folly::Future<std::shared_ptr<RegionLocation>> LocationCache::LocateFromMeta(
-    const TableName &tn, const std::string &row) {
-  return this->LocateMeta()
-      .via(cpu_executor_.get())
-      .then([this](ServerName sn) {
-        // TODO: use RpcClient?
-        auto remote_id = std::make_shared<ConnectionId>(sn.host_name(), sn.port());
-        return this->cp_->GetConnection(remote_id);
-      })
-      .then([tn, row, this](std::shared_ptr<RpcConnection> rpc_connection) {
-        return rpc_connection->SendRequest(std::move(meta_util_.MetaRequest(tn, row)));
-      })
-      .onError([&](const folly::exception_wrapper &ew) {
-        auto promise = InvalidateMeta();
-        throw ew;
-        return static_cast<std::unique_ptr<Response>>(nullptr);
-      })
-      .then([tn, this](std::unique_ptr<Response> resp) {
-        // take the protobuf response and make it into
-        // a region location.
-        return meta_util_.CreateLocation(std::move(*resp), tn);
-      })
-      .then([tn, this](std::shared_ptr<RegionLocation> rl) {
-        // Make sure that the correct location was found.
-        if (rl->region_info().table_name().namespace_() != tn.namespace_() ||
-            rl->region_info().table_name().qualifier() != tn.qualifier()) {
-          throw TableNotFoundException(folly::to<std::string>(tn));
-        }
-        return rl;
-      })
-      .then([this](std::shared_ptr<RegionLocation> rl) {
-        auto remote_id =
-            std::make_shared<ConnectionId>(rl->server_name().host_name(), rl->server_name().port());
-        return rl;
-      })
-      .then([tn, this](std::shared_ptr<RegionLocation> rl) {
-        // now add fetched location to the cache.
-        this->CacheLocation(tn, rl);
-        return rl;
-      });
-}
-
-constexpr const char *MetaUtil::kMetaRegionName;
-
-folly::Future<std::shared_ptr<RegionLocation>> LocationCache::LocateRegion(
-    const TableName &tn, const std::string &row, const RegionLocateType locate_type,
-    const int64_t locate_ns) {
-  // We maybe asked to locate meta itself
-  if (MetaUtil::IsMeta(tn)) {
-    return LocateMeta().then([this](const ServerName &server_name) {
-      auto rl = std::make_shared<RegionLocation>(MetaUtil::kMetaRegionName,
-                                                 meta_util_.meta_region_info(), server_name);
-      return rl;
-    });
-  }
-
-  // TODO: implement region locate type and timeout
-  auto cached_loc = this->GetCachedLocation(tn, row);
-  if (cached_loc != nullptr) {
-    return cached_loc;
-  } else {
-    return this->LocateFromMeta(tn, row);
-  }
-}
-
-// must hold shared lock on locations_lock_
-std::shared_ptr<RegionLocation> LocationCache::GetCachedLocation(const hbase::pb::TableName &tn,
-                                                                 const std::string &row) {
-  auto t_locs = this->GetTableLocations(tn);
-  std::shared_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
-
-  // looking for the "floor" key as a start key
-  auto possible_region = t_locs->upper_bound(row);
-
-  if (t_locs->empty()) {
-    VLOG(5) << "Could not find region in cache, table map is empty";
-    return nullptr;
-  }
-
-  if (possible_region == t_locs->begin()) {
-    VLOG(5) << "Could not find region in cache, all keys are greater, row:" << row
-            << " ,possible_region:" << possible_region->second->DebugString();
-    return nullptr;
-  }
-  --possible_region;
-
-  VLOG(5) << "Found possible region in cache for row:" << row
-          << " ,possible_region:" << possible_region->second->DebugString();
-
-  // found possible start key, now need to check end key
-  if (possible_region->second->region_info().end_key() == "" ||
-      possible_region->second->region_info().end_key() > row) {
-    VLOG(2) << "Found region in cache for row:" << row
-            << " ,region:" << possible_region->second->DebugString();
-    return possible_region->second;
-  } else {
-    return nullptr;
-  }
-}
-
-// must hold unique lock on locations_lock_
-void LocationCache::CacheLocation(const hbase::pb::TableName &tn,
-                                  const std::shared_ptr<RegionLocation> loc) {
-  auto t_locs = this->GetTableLocations(tn);
-  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
-
-  (*t_locs)[loc->region_info().start_key()] = loc;
-  VLOG(1) << "Cached location for region:" << loc->DebugString();
-}
-
-// must hold shared lock on locations_lock_
-bool LocationCache::IsLocationCached(const hbase::pb::TableName &tn, const std::string &row) {
-  return (this->GetCachedLocation(tn, row) != nullptr);
-}
-
-// shared lock needed for cases when this table has been requested before;
-// in the rare case it hasn't, unique lock will be grabbed to add it to cache
-std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetTableLocations(
-    const hbase::pb::TableName &tn) {
-  auto found_locs = this->GetCachedTableLocations(tn);
-  if (found_locs == nullptr) {
-    found_locs = this->GetNewTableLocations(tn);
-  }
-  return found_locs;
-}
-
-std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetCachedTableLocations(
-    const hbase::pb::TableName &tn) {
-  folly::SharedMutexWritePriority::ReadHolder r_holder{locations_lock_};
-
-  auto table_locs = cached_locations_.find(tn);
-  if (table_locs != cached_locations_.end()) {
-    return table_locs->second;
-  } else {
-    return nullptr;
-  }
-}
-
-std::shared_ptr<hbase::PerTableLocationMap> LocationCache::GetNewTableLocations(
-    const hbase::pb::TableName &tn) {
-  // double-check locking under upgradable lock
-  folly::SharedMutexWritePriority::UpgradeHolder u_holder{locations_lock_};
-
-  auto table_locs = cached_locations_.find(tn);
-  if (table_locs != cached_locations_.end()) {
-    return table_locs->second;
-  }
-  folly::SharedMutexWritePriority::WriteHolder w_holder{std::move(u_holder)};
-
-  auto t_locs_p = std::make_shared<std::map<std::string, std::shared_ptr<RegionLocation>>>();
-  cached_locations_.insert(std::make_pair(tn, t_locs_p));
-  return t_locs_p;
-}
-
-// must hold unique lock on locations_lock_
-void LocationCache::ClearCache() {
-  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
-  cached_locations_.clear();
-}
-
-// must hold unique lock on locations_lock_
-void LocationCache::ClearCachedLocations(const hbase::pb::TableName &tn) {
-  VLOG(1) << "ClearCachedLocations, table:" << folly::to<std::string>(tn);
-  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
-  cached_locations_.erase(tn);
-  if (MetaUtil::IsMeta(tn)) {
-    InvalidateMeta();
-  }
-}
-
-// must hold unique lock on locations_lock_
-void LocationCache::ClearCachedLocation(const hbase::pb::TableName &tn, const std::string &row) {
-  VLOG(1) << "ClearCachedLocation, table:" << folly::to<std::string>(tn) << ", row:" << row;
-  auto table_locs = this->GetTableLocations(tn);
-  std::unique_lock<folly::SharedMutexWritePriority> lock(locations_lock_);
-  table_locs->erase(row);
-  if (MetaUtil::IsMeta(tn)) {
-    InvalidateMeta();
-  }
-}
-
-void LocationCache::UpdateCachedLocation(const RegionLocation &loc,
-                                         const folly::exception_wrapper &error) {
-  // TODO: just clears the location for now. We can inspect RegionMovedExceptions, etc later.
-  ClearCachedLocation(loc.region_info().table_name(), loc.region_info().start_key());
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/location-cache.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/location-cache.h b/hbase-native-client/core/location-cache.h
deleted file mode 100644
index 6eb61ef..0000000
--- a/hbase-native-client/core/location-cache.h
+++ /dev/null
@@ -1,219 +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.
- *
- */
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/Executor.h>
-#include <folly/SharedMutex.h>
-#include <folly/futures/Future.h>
-#include <folly/futures/SharedPromise.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-#include <zookeeper/zookeeper.h>
-
-#include <map>
-#include <memory>
-#include <mutex>
-#include <string>
-#include <unordered_map>
-
-#include "connection/connection-pool.h"
-#include "core/async-region-locator.h"
-#include "core/configuration.h"
-#include "core/meta-utils.h"
-#include "core/region-location.h"
-#include "core/zk-util.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-// Forward
-class Request;
-class Response;
-namespace pb {
-class ServerName;
-class TableName;
-}
-
-/** Equals function for TableName (uses namespace and table name) */
-struct TableNameEquals {
-  /** equals */
-  bool operator()(const hbase::pb::TableName &lht, const hbase::pb::TableName &rht) const {
-    return lht.namespace_() == rht.namespace_() && lht.qualifier() == rht.qualifier();
-  }
-};
-
-/** Hash for TableName. */
-struct TableNameHash {
-  /** hash */
-  std::size_t operator()(hbase::pb::TableName const &t) const {
-    std::size_t h = 0;
-    boost::hash_combine(h, t.namespace_());
-    boost::hash_combine(h, t.qualifier());
-    return h;
-  }
-};
-
-// typedefs for location cache
-typedef std::map<std::string, std::shared_ptr<RegionLocation>> PerTableLocationMap;
-typedef std::unordered_map<hbase::pb::TableName, std::shared_ptr<PerTableLocationMap>,
-                           TableNameHash, TableNameEquals>
-    RegionLocationMap;
-
-/**
- * Class that can look up and cache locations.
- */
-class LocationCache : public AsyncRegionLocator {
- public:
-  /**
-   * Constructor.
-   * @param conf Configuration instance to fetch Zookeeper Quorum and Zookeeper Znode.
-   * @param cpu_executor executor used to run non network IO based
-   * continuations.
-   * @param io_executor executor used to talk to the network
-   */
-  LocationCache(std::shared_ptr<hbase::Configuration> conf,
-                std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                std::shared_ptr<ConnectionPool> cp);
-  /**
-   * Destructor.
-   * This will clean up the zookeeper connections.
-   */
-  ~LocationCache();
-
-  /**
-   * Where is meta hosted.
-   *
-   * TODO: This should be a RegionLocation.
-   */
-  folly::Future<hbase::pb::ServerName> LocateMeta();
-
-  /**
-   * Go read meta and find out where a region is located. Most users should
-   * never call this method directly and should use LocateRegion() instead.
-   *
-   * @param tn Table name of the table to look up. This object must live until
-   * after the future is returned
-   *
-   * @param row of the table to look up. This object must live until after the
-   * future is returned
-   */
-  folly::Future<std::shared_ptr<RegionLocation>> LocateFromMeta(const hbase::pb::TableName &tn,
-                                                                const std::string &row);
-
-  /**
-   * The only method clients should use for meta lookups. If corresponding
-   * location is cached, it's returned from the cache, otherwise lookup
-   * in meta table is done, location is cached and then returned.
-   * It's expected that tiny fraction of invocations incurs meta scan.
-   * This method is to look up non-meta regions; use LocateMeta() to get the
-   * location of hbase:meta region.
-   *
-   * @param tn Table name of the table to look up. This object must live until
-   * after the future is returned
-   *
-   * @param row of the table to look up. This object must live until after the
-   * future is returned
-   */
-  folly::Future<std::shared_ptr<RegionLocation>> LocateRegion(
-      const hbase::pb::TableName &tn, const std::string &row,
-      const RegionLocateType locate_type = RegionLocateType::kCurrent,
-      const int64_t locate_ns = 0) override;
-
-  /**
-   * Remove the cached location of meta.
-   */
-  std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> InvalidateMeta();
-
-  /**
-   * Return cached region location corresponding to this row,
-   * nullptr if this location isn't cached.
-   */
-  std::shared_ptr<RegionLocation> GetCachedLocation(const hbase::pb::TableName &tn,
-                                                    const std::string &row);
-
-  /**
-   * Add non-meta region location in the cache (location of meta itself
-   * is cached separately).
-   */
-  void CacheLocation(const hbase::pb::TableName &tn, const std::shared_ptr<RegionLocation> loc);
-
-  /**
-   * Check if location corresponding to this row key is cached.
-   */
-  bool IsLocationCached(const hbase::pb::TableName &tn, const std::string &row);
-
-  /**
-   * Return cached location for all region of this table.
-   */
-  std::shared_ptr<PerTableLocationMap> GetTableLocations(const hbase::pb::TableName &tn);
-
-  /**
-   * Completely clear location cache.
-   */
-  void ClearCache();
-
-  /**
-   * Clear all cached locations for one table.
-   */
-  void ClearCachedLocations(const hbase::pb::TableName &tn);
-
-  /**
-   * Clear cached region location.
-   */
-  void ClearCachedLocation(const hbase::pb::TableName &tn, const std::string &row);
-
-  /**
-   * Update cached region location, possibly using the information from exception.
-   */
-  void UpdateCachedLocation(const RegionLocation &loc,
-                            const folly::exception_wrapper &error) override;
-
-  const std::string &zk_quorum() { return zk_quorum_; }
-
- private:
-  void CloseZooKeeperConnection();
-  void EnsureZooKeeperConnection();
-
- private:
-  void RefreshMetaLocation();
-  hbase::pb::ServerName ReadMetaLocation();
-  std::shared_ptr<RegionLocation> CreateLocation(const Response &resp);
-  std::shared_ptr<hbase::PerTableLocationMap> GetCachedTableLocations(
-      const hbase::pb::TableName &tn);
-  std::shared_ptr<hbase::PerTableLocationMap> GetNewTableLocations(const hbase::pb::TableName &tn);
-
-  /* data */
-  std::shared_ptr<hbase::Configuration> conf_;
-  std::string zk_quorum_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> meta_promise_;
-  std::recursive_mutex meta_lock_;
-  MetaUtil meta_util_;
-  std::shared_ptr<ConnectionPool> cp_;
-
-  // cached region locations
-  RegionLocationMap cached_locations_;
-  folly::SharedMutexWritePriority locations_lock_;
-
-  // TODO: migrate this to a smart pointer with a deleter.
-  zhandle_t *zk_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/meta-utils.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/meta-utils.cc b/hbase-native-client/core/meta-utils.cc
deleted file mode 100644
index 31349a5..0000000
--- a/hbase-native-client/core/meta-utils.cc
+++ /dev/null
@@ -1,125 +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/meta-utils.h"
-
-#include <folly/Conv.h>
-#include <memory>
-#include <utility>
-#include <vector>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "core/response-converter.h"
-#include "exceptions/exception.h"
-#include "if/Client.pb.h"
-#include "serde/region-info.h"
-#include "serde/server-name.h"
-#include "serde/table-name.h"
-
-using hbase::pb::TableName;
-using hbase::pb::RegionInfo;
-using hbase::pb::RegionSpecifier_RegionSpecifierType;
-using hbase::pb::ScanRequest;
-using hbase::pb::ServerName;
-
-namespace hbase {
-
-MetaUtil::MetaUtil() {
-  meta_region_info_.set_start_key("");
-  meta_region_info_.set_end_key("");
-  meta_region_info_.set_offline(false);
-  meta_region_info_.set_split(false);
-  meta_region_info_.set_replica_id(0);
-  meta_region_info_.set_split(false);
-  meta_region_info_.set_region_id(1);
-  meta_region_info_.mutable_table_name()->set_namespace_(MetaUtil::kSystemNamespace);
-  meta_region_info_.mutable_table_name()->set_qualifier(MetaUtil::kMetaTableQualifier);
-}
-
-std::string MetaUtil::RegionLookupRowkey(const TableName &tn, const std::string &row) const {
-  return folly::to<std::string>(tn, ",", row, ",", "999999999999999999");
-}
-
-std::unique_ptr<Request> MetaUtil::MetaRequest(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(MetaUtil::kMetaRegion);
-  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(MetaUtil::kCatalogFamily);
-  info_col->add_qualifier(MetaUtil::kServerColumn);
-  info_col->add_qualifier(MetaUtil::kRegionInfoColumn);
-
-  scan->set_start_row(RegionLookupRowkey(tn, row));
-  return request;
-}
-
-std::shared_ptr<RegionLocation> MetaUtil::CreateLocation(const Response &resp,
-                                                         const TableName &tn) {
-  std::vector<std::shared_ptr<Result>> results = ResponseConverter::FromScanResponse(resp);
-  if (results.size() == 0) {
-    throw TableNotFoundException(folly::to<std::string>(tn));
-  }
-  if (results.size() != 1) {
-    throw std::runtime_error("Was expecting exactly 1 result in meta scan response, got:" +
-                             std::to_string(results.size()));
-  }
-  auto result = *results[0];
-
-  auto region_info_str = result.Value(MetaUtil::kCatalogFamily, MetaUtil::kRegionInfoColumn);
-  auto server_str = result.Value(MetaUtil::kCatalogFamily, MetaUtil::kServerColumn);
-  CHECK(region_info_str);
-  CHECK(server_str);
-
-  auto row = result.Row();
-  auto region_info = folly::to<RegionInfo>(*region_info_str);
-  auto server_name = folly::to<ServerName>(*server_str);
-  return std::make_shared<RegionLocation>(row, std::move(region_info), server_name);
-}
-
-bool MetaUtil::IsMeta(const hbase::pb::TableName &tn) {
-  return folly::to<std::string>(tn) == MetaUtil::kMetaTableName;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/meta-utils.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/meta-utils.h b/hbase-native-client/core/meta-utils.h
deleted file mode 100644
index d178179..0000000
--- a/hbase-native-client/core/meta-utils.h
+++ /dev/null
@@ -1,76 +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.
- *
- */
-#pragma once
-
-#include <memory>
-#include <string>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "core/region-location.h"
-#include "if/HBase.pb.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-
-/**
- * @brief Utility for meta operations.
- */
-class MetaUtil {
- public:
-  static constexpr const char *kSystemNamespace = "hbase";
-  static constexpr const char *kMetaTableQualifier = "meta";
-  static constexpr const char *kMetaTableName = "hbase:meta";
-  static constexpr const char *kMetaRegion = "1588230740";
-  static constexpr const char *kMetaRegionName = "hbase:meta,,1";
-  static constexpr const char *kCatalogFamily = "info";
-  static constexpr const char *kRegionInfoColumn = "regioninfo";
-  static constexpr const char *kServerColumn = "server";
-
-  MetaUtil();
-
-  /**
-   * Given a table and a row give the row key from which to start a scan to find
-   * region locations.
-   */
-  std::string RegionLookupRowkey(const hbase::pb::TableName &tn, const std::string &row) const;
-
-  /**
-   * Given a row we're trying to access create a request to look up the
-   * location.
-   */
-  std::unique_ptr<Request> MetaRequest(const hbase::pb::TableName tn, const std::string &row) const;
-
-  /**
-   * Return a RegionLocation from the parsed Response
-   */
-  std::shared_ptr<RegionLocation> CreateLocation(const Response &resp,
-                                                 const hbase::pb::TableName &tn);
-
-  /**
-   * Return whether the table is the meta table.
-   */
-  static bool IsMeta(const hbase::pb::TableName &tn);
-
-  const pb::RegionInfo &meta_region_info() const { return meta_region_info_; }
-
- private:
-  pb::RegionInfo meta_region_info_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/multi-response.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/multi-response.cc b/hbase-native-client/core/multi-response.cc
deleted file mode 100644
index a4c2108..0000000
--- a/hbase-native-client/core/multi-response.cc
+++ /dev/null
@@ -1,86 +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/multi-response.h"
-#include <glog/logging.h>
-#include "core/region-result.h"
-
-using hbase::pb::RegionLoadStats;
-
-namespace hbase {
-
-MultiResponse::MultiResponse() {}
-
-int MultiResponse::Size() const {
-  int size = 0;
-  for (const auto& result : results_) {
-    size += result.second->ResultOrExceptionSize();
-  }
-  return size;
-}
-
-void MultiResponse::AddRegionResult(const std::string& region_name, int32_t original_index,
-                                    std::shared_ptr<Result> result,
-                                    std::shared_ptr<folly::exception_wrapper> exc) {
-  auto itr = results_.find(region_name);
-  if (itr == results_.end()) {
-    auto region_result = std::make_shared<RegionResult>();
-    region_result->AddResultOrException(original_index, result, exc);
-    results_[region_name] = region_result;
-  } else {
-    itr->second->AddResultOrException(original_index, result, exc);
-  }
-}
-
-void MultiResponse::AddRegionException(const std::string& region_name,
-                                       std::shared_ptr<folly::exception_wrapper> exception) {
-  VLOG(8) << "Store Region Exception:- " << exception->what() << "; Region[" << region_name << "];";
-  bool region_found = false;
-  auto itr = exceptions_.find(region_name);
-  if (itr == exceptions_.end()) {
-    auto region_result = std::make_shared<folly::exception_wrapper>();
-    exceptions_[region_name] = exception;
-  } else {
-    itr->second = exception;
-  }
-}
-
-std::shared_ptr<folly::exception_wrapper> MultiResponse::RegionException(
-    const std::string& region_name) const {
-  auto find = exceptions_.at(region_name);
-  return find;
-}
-
-const std::map<std::string, std::shared_ptr<folly::exception_wrapper> >&
-MultiResponse::RegionExceptions() const {
-  return exceptions_;
-}
-
-void MultiResponse::AddStatistic(const std::string& region_name,
-                                 std::shared_ptr<RegionLoadStats> stat) {
-  results_[region_name]->set_stat(stat);
-}
-
-const std::map<std::string, std::shared_ptr<RegionResult> >& MultiResponse::RegionResults() const {
-  return results_;
-}
-
-MultiResponse::~MultiResponse() {}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/multi-response.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/multi-response.h b/hbase-native-client/core/multi-response.h
deleted file mode 100644
index d38cfd6..0000000
--- a/hbase-native-client/core/multi-response.h
+++ /dev/null
@@ -1,79 +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.
- *
- */
-
-#pragma once
-
-#include <core/region-result.h>
-#include <folly/ExceptionWrapper.h>
-#include <exception>
-#include <map>
-#include <memory>
-#include <string>
-
-#include "core/result.h"
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-class MultiResponse {
- public:
-  MultiResponse();
-  /**
-   * @brief Returns Number of pairs in this container
-   */
-  int Size() const;
-
-  /**
-   * Add the pair to the container, grouped by the regionName
-   *
-   * @param regionName
-   * @param originalIndex the original index of the Action (request).
-   * @param resOrEx the result or error; will be empty for successful Put and Delete actions.
-   */
-  void AddRegionResult(const std::string& region_name, int32_t original_index,
-                       std::shared_ptr<Result> result,
-                       std::shared_ptr<folly::exception_wrapper> exc);
-
-  void AddRegionException(const std::string& region_name,
-                          std::shared_ptr<folly::exception_wrapper> exception);
-
-  /**
-   * @return the exception for the region, if any. Null otherwise.
-   */
-  std::shared_ptr<folly::exception_wrapper> RegionException(const std::string& region_name) const;
-
-  const std::map<std::string, std::shared_ptr<folly::exception_wrapper>>& RegionExceptions() const;
-
-  void AddStatistic(const std::string& region_name, std::shared_ptr<pb::RegionLoadStats> stat);
-
-  const std::map<std::string, std::shared_ptr<RegionResult>>& RegionResults() const;
-
-  ~MultiResponse();
-
- private:
-  // map of regionName to map of Results by the original index for that Result
-  std::map<std::string, std::shared_ptr<RegionResult>> results_;
-  /**
-   * The server can send us a failure for the region itself, instead of individual failure.
-   * It's a part of the protobuf definition.
-   */
-  std::map<std::string, std::shared_ptr<folly::exception_wrapper>> exceptions_;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/mutation.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/mutation.cc b/hbase-native-client/core/mutation.cc
deleted file mode 100644
index 7182202..0000000
--- a/hbase-native-client/core/mutation.cc
+++ /dev/null
@@ -1,69 +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/mutation.h"
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-
-namespace hbase {
-
-Mutation::Mutation(const std::string &row) : Row(row) {}
-Mutation::Mutation(const std::string &row, int64_t timestamp) : Row(row), timestamp_(timestamp) {}
-
-Mutation::Mutation(const Mutation &mutation) {
-  row_ = mutation.row_;
-  durability_ = mutation.durability_;
-  timestamp_ = mutation.timestamp_;
-  for (auto const &e : mutation.family_map_) {
-    for (auto const &c : e.second) {
-      family_map_[e.first].push_back(std::make_unique<Cell>(*c));
-    }
-  }
-}
-
-Mutation &Mutation::operator=(const Mutation &mutation) {
-  row_ = mutation.row_;
-  durability_ = mutation.durability_;
-  timestamp_ = mutation.timestamp_;
-  for (auto const &e : mutation.family_map_) {
-    for (auto const &c : e.second) {
-      family_map_[e.first].push_back(std::make_unique<Cell>(*c));
-    }
-  }
-  return *this;
-}
-
-pb::MutationProto_Durability Mutation::Durability() const { return durability_; }
-
-Mutation &Mutation::SetDurability(pb::MutationProto_Durability durability) {
-  durability_ = durability;
-  return *this;
-}
-
-bool Mutation::HasFamilies() const { return !family_map_.empty(); }
-
-std::unique_ptr<Cell> Mutation::CreateCell(const std::string &family, const std::string &qualifier,
-                                           int64_t timestamp, const std::string &value) {
-  return std::make_unique<Cell>(row_, family, qualifier, timestamp, value, hbase::CellType::PUT);
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/mutation.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/mutation.h b/hbase-native-client/core/mutation.h
deleted file mode 100644
index 496891e..0000000
--- a/hbase-native-client/core/mutation.h
+++ /dev/null
@@ -1,96 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <limits>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/cell.h"
-#include "core/row.h"
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-class Mutation : public Row {
- public:
-  /**
-   * Constructors
-   */
-  explicit Mutation(const std::string& row);
-  Mutation(const std::string& row, int64_t timestamp);
-  Mutation(const Mutation& cmutation);
-  Mutation& operator=(const Mutation& cmutation);
-
-  virtual ~Mutation() = default;
-
-  /**
-   * @brief Returns the Mutation family map for this operation.
-   */
-  const std::map<std::string, std::vector<std::unique_ptr<Cell>>>& FamilyMap() const {
-    return family_map_;
-  }
-
-  /**
-   * @brief Returns the timerange for this Get
-   */
-  int64_t TimeStamp() const { return timestamp_; }
-
-  /**
-   * @brief Get versions of columns with the specified timestamp.
-   * @param The timestamp to be set
-   */
-  Mutation& SetTimeStamp(int64_t timestamp) {
-    timestamp_ = timestamp;
-    return *this;
-  }
-
-  /**
-   * @brief Returns true if family map is non empty false otherwise
-   */
-  bool HasFamilies() const;
-
-  /**
-   * @brief Returns the durability level for this Mutation operation
-   */
-  pb::MutationProto_Durability Durability() const;
-
-  /**
-   * @brief Sets the durability level for this Mutation operation
-   * @param durability  the durability to be set
-   */
-  Mutation& SetDurability(pb::MutationProto_Durability durability);
-
- public:
-  static const constexpr int64_t kLatestTimestamp = std::numeric_limits<int64_t>::max();
-
- protected:
-  std::map<std::string, std::vector<std::unique_ptr<Cell>>> family_map_;
-  pb::MutationProto_Durability durability_ =
-      hbase::pb::MutationProto_Durability::MutationProto_Durability_USE_DEFAULT;
-  int64_t timestamp_ = kLatestTimestamp;
-
-  std::unique_ptr<Cell> CreateCell(const std::string& family, const std::string& qualifier,
-                                   int64_t timestamp, const std::string& value);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/put-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put-test.cc b/hbase-native-client/core/put-test.cc
deleted file mode 100644
index d4ed00a..0000000
--- a/hbase-native-client/core/put-test.cc
+++ /dev/null
@@ -1,135 +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 <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include "core/mutation.h"
-#include "core/put.h"
-#include "utils/time-util.h"
-
-using hbase::Put;
-using hbase::Cell;
-using hbase::CellType;
-using hbase::Mutation;
-using hbase::TimeUtil;
-
-const constexpr int64_t Mutation::kLatestTimestamp;
-
-TEST(Put, Row) {
-  Put put{"foo"};
-  EXPECT_EQ("foo", put.row());
-}
-
-TEST(Put, Durability) {
-  Put put{"row"};
-  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, put.Durability());
-
-  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
-  put.SetDurability(skipWal);
-  EXPECT_EQ(skipWal, put.Durability());
-}
-
-TEST(Put, Timestamp) {
-  Put put{"row"};
-
-  // test default timestamp
-  EXPECT_EQ(Mutation::kLatestTimestamp, put.TimeStamp());
-
-  // set custom timestamp
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  put.SetTimeStamp(ts);
-  EXPECT_EQ(ts, put.TimeStamp());
-
-  // Add a column with custom timestamp
-  put.AddColumn("f", "q", "v");
-  auto &cell = put.FamilyMap().at("f")[0];
-  EXPECT_EQ(ts, cell->Timestamp());
-}
-
-TEST(Put, HasFamilies) {
-  Put put{"row"};
-
-  EXPECT_EQ(false, put.HasFamilies());
-
-  put.AddColumn("f", "q", "v");
-  EXPECT_EQ(true, put.HasFamilies());
-}
-
-TEST(Put, Add) {
-  CellType cell_type = CellType::PUT;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-
-  // add first cell
-  Put put{"row"};
-  put.Add(std::move(cell));
-  EXPECT_EQ(1, put.FamilyMap().size());
-  EXPECT_EQ(1, put.FamilyMap().at(family).size());
-
-  // add a non-matching row
-  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-  Put put2{"foo"};
-  ASSERT_THROW(put2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
-
-  // add a second cell with same family
-  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
-  put.Add(std::move(cell3));
-  EXPECT_EQ(1, put.FamilyMap().size());
-  EXPECT_EQ(2, put.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
-  put.Add(std::move(cell4));
-  EXPECT_EQ(2, put.FamilyMap().size());
-  EXPECT_EQ(1, put.FamilyMap().at("family-2").size());
-}
-
-TEST(Put, AddColumn) {
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-
-  Put put{"row"};
-  put.AddColumn(family, column, value);
-  EXPECT_EQ(1, put.FamilyMap().size());
-  EXPECT_EQ(1, put.FamilyMap().at(family).size());
-
-  // add a second cell with same family
-  put.AddColumn(family, "column-2", value);
-  EXPECT_EQ(1, put.FamilyMap().size());
-  EXPECT_EQ(2, put.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  put.AddColumn("family-2", column, value);
-  EXPECT_EQ(2, put.FamilyMap().size());
-  EXPECT_EQ(1, put.FamilyMap().at("family-2").size());
-
-  // use the AddColumn overload
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  put.AddColumn(family, column, ts, value);
-  EXPECT_EQ(2, put.FamilyMap().size());
-  EXPECT_EQ(3, put.FamilyMap().at(family).size());
-  auto &cell = put.FamilyMap().at(family)[2];
-  EXPECT_EQ(ts, cell->Timestamp());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/put.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put.cc b/hbase-native-client/core/put.cc
deleted file mode 100644
index bb20c5c..0000000
--- a/hbase-native-client/core/put.cc
+++ /dev/null
@@ -1,68 +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/put.h"
-#include <folly/Conv.h>
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-#include <utility>
-
-namespace hbase {
-
-/**
- *  @brief Add the specified column and value to this Put operation.
- *  @param family family name
- *  @param qualifier column qualifier
- *  @param value column value
- */
-Put& Put::AddColumn(const std::string& family, const std::string& qualifier,
-                    const std::string& value) {
-  return AddColumn(family, qualifier, timestamp_, value);
-}
-
-/**
- *  @brief Add the specified column and value to this Put operation.
- *  @param family family name
- *  @param qualifier column qualifier
- *  @param timestamp version timestamp
- *  @param value column value
- */
-Put& Put::AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp,
-                    const std::string& value) {
-  if (timestamp < 0) {
-    throw std::runtime_error("Timestamp cannot be negative. ts=" +
-                             folly::to<std::string>(timestamp));
-  }
-
-  return Add(CreateCell(family, qualifier, timestamp, value));
-}
-
-Put& Put::Add(std::unique_ptr<Cell> cell) {
-  if (cell->Row() != row_) {
-    throw std::runtime_error("The row in " + cell->DebugString() +
-                             " doesn't match the original one " + row_);
-  }
-
-  family_map_[cell->Family()].push_back(std::move(cell));
-  return *this;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/put.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put.h b/hbase-native-client/core/put.h
deleted file mode 100644
index 8a43dec..0000000
--- a/hbase-native-client/core/put.h
+++ /dev/null
@@ -1,68 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/cell.h"
-#include "core/mutation.h"
-
-namespace hbase {
-
-class Put : public Mutation {
- public:
-  /**
-   * Constructors
-   */
-  explicit Put(const std::string& row) : Mutation(row) {}
-  Put(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
-  Put(const Put& cput) : Mutation(cput) {}
-  Put& operator=(const Put& cput) {
-    Mutation::operator=(cput);
-    return *this;
-  }
-
-  ~Put() = default;
-
-  /**
-   *  @brief Add the specified column and value to this Put operation.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param value column value
-   */
-  Put& AddColumn(const std::string& family, const std::string& qualifier, const std::string& value);
-
-  /**
-   *  @brief Add the specified column and value to this Put operation.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param timestamp version timestamp
-   *  @param value column value
-   */
-  Put& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp,
-                 const std::string& value);
-
-  Put& Add(std::unique_ptr<Cell> cell);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/query.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/query.h b/hbase-native-client/core/query.h
deleted file mode 100644
index 301f448..0000000
--- a/hbase-native-client/core/query.h
+++ /dev/null
@@ -1,54 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-
-#include "core/filter.h"
-
-namespace hbase {
-
-/**
- * Base class for read RPC calls (Get / Scan).
- */
-class Query {
- public:
-  Query() = default;
-  Query(const Query &query) {
-    // filter can be a custom subclass of Filter, so we do not do a deep copy here.
-    filter_ = query.filter_;
-  }
-
-  Query &operator=(const Query &query) {
-    filter_ = query.filter_;
-    return *this;
-  }
-
-  virtual ~Query() {}
-
-  void SetFilter(std::shared_ptr<Filter> filter) { filter_ = filter; }
-
-  const std::shared_ptr<Filter> filter() const { return filter_; }
-
- protected:
-  std::shared_ptr<Filter> filter_ = nullptr;
-};
-
-}  // namespace hbase


[11/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/response.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/response.h b/hbase-native-client/include/hbase/connection/response.h
new file mode 100644
index 0000000..5fe8b79
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/response.h
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Conv.h>
+#include <folly/ExceptionWrapper.h>
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "hbase/serde/cell-scanner.h"
+
+// Forward
+namespace google {
+namespace protobuf {
+class Message;
+}
+}
+
+namespace hbase {
+
+/**
+ * @brief Class representing a rpc response
+ *
+ * This is the class sent to a service.
+ */
+class Response {
+ public:
+  /**
+   * Constructor.
+   * Initinalizes the call id to 0. 0 should never be a valid call id.
+   */
+  Response() : call_id_(0), resp_msg_(nullptr), cell_scanner_(nullptr), exception_(nullptr) {}
+
+  /** Get the call_id */
+  uint32_t call_id() { return call_id_; }
+
+  /** Set the call_id */
+  void set_call_id(uint32_t call_id) { call_id_ = call_id; }
+
+  /**
+   * Get the response message.
+   * The caller is reponsible for knowing the type. In practice the call id is
+   * used to figure out the type.
+   */
+  std::shared_ptr<google::protobuf::Message> resp_msg() const { return resp_msg_; }
+
+  /** Set the response message. */
+  void set_resp_msg(std::shared_ptr<google::protobuf::Message> response) {
+    resp_msg_ = std::move(response);
+  }
+
+  void set_cell_scanner(std::shared_ptr<CellScanner> cell_scanner) { cell_scanner_ = cell_scanner; }
+
+  const std::shared_ptr<CellScanner> cell_scanner() const { return cell_scanner_; }
+
+  folly::exception_wrapper exception() { return exception_; }
+
+  void set_exception(folly::exception_wrapper value) { exception_ = value; }
+
+  std::string DebugString() const {
+    std::string s{"call_id:"};
+    s += folly::to<std::string>(call_id_);
+    s += ", resp_msg:";
+    s += resp_msg_->ShortDebugString();
+    return s;
+  }
+
+ private:
+  uint32_t call_id_;
+  std::shared_ptr<google::protobuf::Message> resp_msg_;
+  std::shared_ptr<CellScanner> cell_scanner_;
+  folly::exception_wrapper exception_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-client.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-client.h b/hbase-native-client/include/hbase/connection/rpc-client.h
new file mode 100644
index 0000000..1189410
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-client.h
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <google/protobuf/service.h>
+
+#include <folly/ExceptionWrapper.h>
+#include <chrono>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "hbase/connection/connection-id.h"
+#include "hbase/connection/connection-pool.h"
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/security/user.h"
+
+namespace hbase {
+
+class RpcClient {
+ public:
+  RpcClient(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+            std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+            std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
+            std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
+
+  virtual ~RpcClient() { Close(); }
+
+  virtual std::unique_ptr<Response> SyncCall(const std::string &host, uint16_t port,
+                                             std::unique_ptr<Request> req,
+                                             std::shared_ptr<security::User> ticket);
+
+  virtual std::unique_ptr<Response> SyncCall(const std::string &host, uint16_t port,
+                                             std::unique_ptr<Request> req,
+                                             std::shared_ptr<security::User> ticket,
+                                             const std::string &service_name);
+
+  virtual folly::Future<std::unique_ptr<Response>> AsyncCall(
+      const std::string &host, uint16_t port, std::unique_ptr<Request> req,
+      std::shared_ptr<security::User> ticket);
+
+  virtual folly::Future<std::unique_ptr<Response>> AsyncCall(const std::string &host, uint16_t port,
+                                                             std::unique_ptr<Request> req,
+                                                             std::shared_ptr<security::User> ticket,
+                                                             const std::string &service_name);
+
+  virtual void Close();
+
+  std::shared_ptr<ConnectionPool> connection_pool() const { return cp_; }
+
+ private:
+  std::shared_ptr<RpcConnection> GetConnection(std::shared_ptr<ConnectionId> remote_id);
+  folly::Future<std::unique_ptr<Response>> SendRequest(std::shared_ptr<ConnectionId> remote_id,
+                                                       std::unique_ptr<Request> req);
+  template <typename EXCEPTION>
+  folly::Future<std::unique_ptr<Response>> GetFutureWithException(const EXCEPTION &e);
+
+  folly::Future<std::unique_ptr<Response>> GetFutureWithException(
+      const folly::exception_wrapper &ew);
+
+ private:
+  std::shared_ptr<ConnectionPool> cp_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<Configuration> conf_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-connection.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-connection.h b/hbase-native-client/include/hbase/connection/rpc-connection.h
new file mode 100644
index 0000000..cbbac63
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-connection.h
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <mutex>
+#include <utility>
+
+#include "hbase/connection/connection-factory.h"
+#include "hbase/connection/connection-id.h"
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/connection/service.h"
+
+namespace hbase {
+
+class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
+ public:
+  RpcConnection(std::shared_ptr<ConnectionId> connection_id, std::shared_ptr<ConnectionFactory> cf)
+      : connection_id_(connection_id), cf_(cf), hbase_service_(nullptr) {}
+
+  virtual ~RpcConnection() { Close(); }
+
+  virtual std::shared_ptr<ConnectionId> remote_id() const { return connection_id_; }
+
+  virtual folly::Future<std::unique_ptr<Response>> SendRequest(std::unique_ptr<Request> req) {
+    std::lock_guard<std::recursive_mutex> lock(mutex_);
+    if (hbase_service_ == nullptr) {
+      Connect();
+    }
+    VLOG(5) << "Calling RpcConnection::SendRequest()";  // TODO
+    return (*hbase_service_)(std::move(req));
+  }
+
+  virtual void Close() {
+    std::lock_guard<std::recursive_mutex> lock(mutex_);
+    if (hbase_service_) {
+      hbase_service_->close();
+      hbase_service_ = nullptr;
+    }
+    if (client_bootstrap_) {
+      client_bootstrap_ = nullptr;
+    }
+  }
+
+ private:
+  void Connect() {
+    client_bootstrap_ = cf_->MakeBootstrap();
+    auto dispatcher = cf_->Connect(shared_from_this(), client_bootstrap_, remote_id()->host(),
+                                   remote_id()->port());
+    hbase_service_ = std::move(dispatcher);
+  }
+
+ private:
+  std::recursive_mutex mutex_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<ConnectionId> connection_id_;
+  std::shared_ptr<HBaseService> hbase_service_;
+  std::shared_ptr<ConnectionFactory> cf_;
+  std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-fault-injector-inl.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-fault-injector-inl.h b/hbase-native-client/include/hbase/connection/rpc-fault-injector-inl.h
new file mode 100644
index 0000000..8bbaddf
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-fault-injector-inl.h
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+namespace hbase {
+
+template <typename T>
+std::shared_ptr<T> RpcFaultInjector<T>::instance = std::make_shared<T>();
+
+template <typename T>
+RpcFaultInjector<T>::RpcFaultInjector() {}
+
+template <typename T>
+RpcFaultInjector<T>::~RpcFaultInjector() {}
+
+template <typename T>
+std::shared_ptr<T> RpcFaultInjector<T>::Get() {
+  return instance;
+}
+
+template <typename T>
+void RpcFaultInjector<T>::Set(std::shared_ptr<T> injector) {
+  instance = injector;
+}
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-fault-injector.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-fault-injector.h b/hbase-native-client/include/hbase/connection/rpc-fault-injector.h
new file mode 100644
index 0000000..1d21875
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-fault-injector.h
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/io/async/AsyncTransport.h>
+#include "hbase/connection/pipeline.h"
+
+namespace hbase {
+
+template <typename T>
+class RpcFaultInjector {
+ public:
+  RpcFaultInjector();
+  virtual ~RpcFaultInjector();
+
+  static std::shared_ptr<T> Get();
+  static void Set(std::shared_ptr<T> instance);
+
+ private:
+  static std::shared_ptr<T> instance;
+};
+
+class RpcClientFaultInjector : public RpcFaultInjector<RpcClientFaultInjector> {
+ public:
+  RpcClientFaultInjector() {}
+  virtual ~RpcClientFaultInjector() {}
+  /**
+   * Here goes virtual functions for injecting various faults. They should be no-ops by default.
+   * Sub classes of RpcClientFaultInjector will override by providing concrete faults.
+   */
+};
+} /* namespace hbase */
+
+#include "hbase/connection/rpc-fault-injector-inl.h"

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-test-server-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-test-server-handler.h b/hbase-native-client/include/hbase/connection/rpc-test-server-handler.h
new file mode 100644
index 0000000..ef6ed4e
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-test-server-handler.h
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <wangle/channel/Handler.h>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/serde/rpc-serde.h"
+
+using namespace hbase;
+
+namespace hbase {
+// A real rpc server would probably use generated client/server stubs
+class RpcTestServerSerializeHandler
+    : public wangle::Handler<std::unique_ptr<folly::IOBuf>, std::unique_ptr<Request>,
+                             std::unique_ptr<Response>, std::unique_ptr<folly::IOBuf>> {
+ public:
+  RpcTestServerSerializeHandler() : serde_() {}
+
+  void read(Context* ctx, std::unique_ptr<folly::IOBuf> buf) override;
+
+  folly::Future<folly::Unit> write(Context* ctx, std::unique_ptr<Response> resp) override;
+
+ private:
+  std::unique_ptr<Request> CreateReceivedRequest(const std::string& method_name);
+
+ private:
+  hbase::RpcSerde serde_;
+};
+}  // end of namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/rpc-test-server.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/rpc-test-server.h b/hbase-native-client/include/hbase/connection/rpc-test-server.h
new file mode 100644
index 0000000..76ab8ba
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/rpc-test-server.h
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+#include <folly/SocketAddress.h>
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+#include <wangle/service/ExecutorFilter.h>
+#include <wangle/service/Service.h>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/exceptions/exception.h"
+
+using namespace hbase;
+using namespace folly;
+using namespace wangle;
+
+namespace hbase {
+using RpcTestServerSerializePipeline = wangle::Pipeline<IOBufQueue&, std::unique_ptr<Response>>;
+
+class RpcTestException : public IOException {
+ public:
+  RpcTestException() {}
+  RpcTestException(const std::string& what) : IOException(what) {}
+  RpcTestException(const std::string& what, const folly::exception_wrapper& cause)
+      : IOException(what, cause) {}
+  RpcTestException(const folly::exception_wrapper& cause) : IOException("", cause) {}
+};
+
+class RpcTestService : public Service<std::unique_ptr<Request>, std::unique_ptr<Response>> {
+ public:
+  RpcTestService(std::shared_ptr<folly::SocketAddress> socket_address)
+      : socket_address_(socket_address) {}
+  virtual ~RpcTestService() = default;
+  Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> request) override;
+
+ private:
+  std::shared_ptr<folly::SocketAddress> socket_address_;
+};
+
+class RpcTestServerPipelineFactory : public PipelineFactory<RpcTestServerSerializePipeline> {
+ public:
+  RpcTestServerSerializePipeline::Ptr newPipeline(
+      std::shared_ptr<AsyncTransportWrapper> sock) override;
+
+ private:
+  void initService(std::shared_ptr<AsyncTransportWrapper> sock);
+
+ private:
+  std::shared_ptr<ExecutorFilter<std::unique_ptr<Request>, std::unique_ptr<Response>>> service_{
+      nullptr};
+};
+}  // end of namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/sasl-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/sasl-handler.h b/hbase-native-client/include/hbase/connection/sasl-handler.h
new file mode 100644
index 0000000..bb7f294
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/sasl-handler.h
@@ -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.
+ *
+ */
+#pragma once
+
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+#include <sasl/saslutil.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/connection/sasl-util.h"
+#include "hbase/connection/service.h"
+#include "hbase/security/user.h"
+#include "hbase/serde/rpc-serde.h"
+
+namespace hbase {
+
+/**
+ * Class to perform SASL handshake with server (currently works with regionserver principals only)
+ * It is inserted between EventBaseHandler and LengthFieldBasedFrameDecoder in the pipeline
+ * SaslHandler would intercept writes to server by buffering the IOBuf's and start the handshake
+ * process
+ *   (via sasl_client_XX calls provided by Cyrus)
+ * After handshake is complete, SaslHandler would send the buffered IOBuf's to server and
+ *   act as pass-thru from then on
+ */
+class SaslHandler
+    : public wangle::HandlerAdapter<folly::IOBufQueue&, std::unique_ptr<folly::IOBuf>> {
+ public:
+  explicit SaslHandler(std::string user_name, std::shared_ptr<Configuration> conf);
+  SaslHandler(const SaslHandler& hdlr);
+  ~SaslHandler();
+
+  // from HandlerAdapter
+  void read(Context* ctx, folly::IOBufQueue& buf) override;
+  folly::Future<folly::Unit> write(Context* ctx, std::unique_ptr<folly::IOBuf> buf) override;
+  void transportActive(Context* ctx) override;
+
+ private:
+  // used by Cyrus
+  sasl_conn_t* sconn_ = nullptr;
+  std::string user_name_;
+  std::string service_name_;
+  std::string host_name_;
+  bool secure_;
+  std::atomic_flag sasl_connection_setup_started_;
+  std::atomic<bool> sasl_connection_setup_in_progress_{true};
+  // vector of folly::IOBuf which buffers client writes before handshake is complete
+  std::vector<std::unique_ptr<folly::IOBuf>> iobuf_;
+  SaslUtil sasl_util_;
+
+  // writes the output returned by sasl_client_XX to server
+  folly::Future<folly::Unit> WriteSaslOutput(Context* ctx, const char* out, unsigned int outlen);
+  folly::Future<folly::Unit> SaslInit(Context* ctx);
+  void FinishAuth(Context* ctx, folly::IOBufQueue* bufQueue);
+  void ContinueSaslNegotiation(Context* ctx, folly::IOBufQueue* buf);
+  std::string ParseServiceName(std::shared_ptr<Configuration> conf, bool secure);
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/sasl-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/sasl-util.h b/hbase-native-client/include/hbase/connection/sasl-util.h
new file mode 100644
index 0000000..4d236df
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/sasl-util.h
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <mutex>
+#include <string>
+
+#include "hbase/client/configuration.h"
+
+class SaslUtil {
+ public:
+  void InitializeSaslLib(void);
+  static std::string ParseServiceName(std::shared_ptr<hbase::Configuration> conf, bool secure);
+
+ private:
+  static constexpr const char *kDefaultPluginDir = "/usr/lib/sasl2";
+  // for now the sasl handler is hardcoded to work against the regionservers only. In the future, if
+  // we
+  // need the master rpc to work, we could have a map of service names to principals to use (similar
+  // to the Java implementation)
+  static constexpr const char *kServerPrincipalConfKey = "hbase.regionserver.kerberos.principal";
+
+  static int GetPluginPath(void *context, const char **path);
+  static void *MutexNew(void);
+  static int MutexLock(void *m);
+  static int MutexUnlock(void *m);
+  static void MutexDispose(void *m);
+  static std::once_flag library_inited_;
+};

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/service.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/service.h b/hbase-native-client/include/hbase/connection/service.h
new file mode 100644
index 0000000..a70fd6b
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/service.h
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <wangle/service/Service.h>
+
+#include <memory>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+
+namespace hbase {
+using HBaseService = wangle::Service<std::unique_ptr<Request>, std::unique_ptr<Response>>;
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/exceptions/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/exceptions/BUCK b/hbase-native-client/include/hbase/exceptions/BUCK
new file mode 100644
index 0000000..ee296ff
--- /dev/null
+++ b/hbase-native-client/include/hbase/exceptions/BUCK
@@ -0,0 +1,28 @@
+##
+# 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.
+
+cxx_library(
+    name="exceptions",
+    header_namespace="hbase/exceptions",
+    exported_headers=[
+        "exception.h",
+    ],
+    deps=[
+        "//third-party:folly",
+    ],
+    compiler_flags=['-Weffc++'],
+    visibility=['//src/hbase/exceptions/...', '//include/hbase/client/...', '//include/hbase/connection/...'],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/exceptions/exception.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/exceptions/exception.h b/hbase-native-client/include/hbase/exceptions/exception.h
new file mode 100644
index 0000000..bc3b291
--- /dev/null
+++ b/hbase-native-client/include/hbase/exceptions/exception.h
@@ -0,0 +1,328 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/io/IOBuf.h>
+#include <exception>
+#include <memory>
+#include <string>
+#include <vector>
+
+namespace hbase {
+
+class ThrowableWithExtraContext {
+ public:
+  ThrowableWithExtraContext(folly::exception_wrapper cause, const int64_t& when)
+      : cause_(cause), when_(when), extras_("") {}
+
+  ThrowableWithExtraContext(folly::exception_wrapper cause, const int64_t& when,
+                            const std::string& extras)
+      : cause_(cause), when_(when), extras_(extras) {}
+
+  virtual std::string ToString() {
+    // TODO:
+    // return new Date(this.when).toString() + ", " + extras + ", " + t.toString();
+    return extras_ + ", " + cause_.what().toStdString();
+  }
+
+  virtual folly::exception_wrapper cause() { return cause_; }
+
+ private:
+  folly::exception_wrapper cause_;
+  int64_t when_;
+  std::string extras_;
+};
+
+class IOException : public std::logic_error {
+ public:
+  IOException() : logic_error(""), do_not_retry_(false) {}
+
+  explicit IOException(const std::string& what) : logic_error(what), do_not_retry_(false) {}
+
+  IOException(const std::string& what, bool do_not_retry)
+      : logic_error(what), do_not_retry_(do_not_retry) {}
+
+  IOException(const std::string& what, const folly::exception_wrapper& cause)
+      : logic_error(what), cause_(cause), do_not_retry_(false) {}
+
+  IOException(const std::string& what, folly::exception_wrapper cause, bool do_not_retry)
+      : logic_error(what), cause_(cause), do_not_retry_(do_not_retry) {}
+
+  virtual ~IOException() = default;
+
+  virtual folly::exception_wrapper cause() const { return cause_; }
+
+  bool do_not_retry() const { return do_not_retry_; }
+
+  IOException* set_do_not_retry(bool value) {
+    do_not_retry_ = value;
+    return this;
+  }
+
+ private:
+  folly::exception_wrapper cause_;
+  // In case the exception is a RemoteException, do_not_retry information can come from
+  // the PB field in the RPC response, or it can be deduced from the Java-exception
+  // hierarchy in ExceptionUtil::ShouldRetry(). In case this is a client-side exception
+  // raised from the C++ internals, set this field so that the retrying callers can
+  // re-throw the exception without retrying.
+  bool do_not_retry_;
+};
+
+class RetriesExhaustedException : public IOException {
+ public:
+  RetriesExhaustedException(const int& num_retries,
+                            std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions)
+      : IOException(GetMessage(num_retries, exceptions),
+                    exceptions->empty() ? folly::exception_wrapper{}
+                                        : (*exceptions)[exceptions->size() - 1].cause()),
+        num_retries_(num_retries) {}
+  virtual ~RetriesExhaustedException() = default;
+
+  int32_t num_retries() const { return num_retries_; }
+
+ private:
+  std::string GetMessage(const int& num_retries,
+                         std::shared_ptr<std::vector<ThrowableWithExtraContext>> exceptions) {
+    std::string buffer("Failed after attempts=");
+    buffer.append(std::to_string(num_retries + 1));
+    buffer.append(", exceptions:\n");
+    for (auto it = exceptions->begin(); it != exceptions->end(); it++) {
+      buffer.append(it->ToString());
+      buffer.append("\n");
+    }
+    return buffer;
+  }
+
+ private:
+  int32_t num_retries_;
+};
+
+class ConnectionException : public IOException {
+ public:
+  ConnectionException() {}
+
+  ConnectionException(const std::string& what) : IOException(what) {}
+
+  ConnectionException(const folly::exception_wrapper& cause) : IOException("", cause) {}
+
+  ConnectionException(const std::string& what, const folly::exception_wrapper& cause)
+      : IOException(what, cause) {}
+};
+
+class RemoteException : public IOException {
+ public:
+  RemoteException() : IOException(), port_(0) {}
+
+  explicit RemoteException(const std::string& what) : IOException(what), port_(0) {}
+
+  RemoteException(const std::string& what, folly::exception_wrapper cause)
+      : IOException(what, cause), port_(0) {}
+
+  virtual ~RemoteException() = default;
+
+  std::string exception_class_name() const { return exception_class_name_; }
+
+  RemoteException* set_exception_class_name(const std::string& value) {
+    exception_class_name_ = value;
+    return this;
+  }
+
+  std::string stack_trace() const { return stack_trace_; }
+
+  RemoteException* set_stack_trace(const std::string& value) {
+    stack_trace_ = value;
+    return this;
+  }
+
+  std::string hostname() const { return hostname_; }
+
+  RemoteException* set_hostname(const std::string& value) {
+    hostname_ = value;
+    return this;
+  }
+
+  int port() const { return port_; }
+
+  RemoteException* set_port(int value) {
+    port_ = value;
+    return this;
+  }
+
+ private:
+  std::string exception_class_name_;
+  std::string stack_trace_;
+  std::string hostname_;
+  int port_;
+};
+
+/**
+ * Raised from the client side if we cannot find the table (does not have anything to
+ * do with the Java exception of the same name).
+ */
+class TableNotFoundException : public IOException {
+ public:
+  explicit TableNotFoundException(const std::string& table_name)
+      : IOException("Table cannot be found:" + table_name, true) {}
+
+  virtual ~TableNotFoundException() = default;
+};
+
+/**
+ * List of known exceptions from Java side, and Java-specific exception logic
+ */
+class ExceptionUtil {
+ private:
+  // unknown scanner and sub-classes
+  static constexpr const char* kUnknownScannerException =
+      "org.apache.hadoop.hbase.UnknownScannerException";
+
+  // not serving region and sub-classes
+  static constexpr const char* kNotServingRegionException =
+      "org.apache.hadoop.hbase.NotServingRegionException";
+  static constexpr const char* kRegionInRecoveryException =
+      "org.apache.hadoop.hbase.exceptions.RegionInRecoveryException";
+  static constexpr const char* kRegionOpeningException =
+      "org.apache.hadoop.hbase.exceptions.RegionOpeningException";
+  static constexpr const char* kRegionMovedException =
+      "org.apache.hadoop.hbase.exceptions.RegionMovedException";
+
+  // Region server stopped and sub-classes
+  static constexpr const char* kRegionServerStoppedException =
+      "org.apache.hadoop.hbase.regionserver.RegionServerStoppedException";
+  static constexpr const char* kRegionServerAbortedException =
+      "org.apache.hadoop.hbase.regionserver.RegionServerAbortedException";
+
+  // other scanner related exceptions
+  static constexpr const char* kOutOfOrderScannerNextException =
+      "org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException";
+  static constexpr const char* kScannerResetException =
+      "org.apache.hadoop.hbase.exceptions.ScannerResetException";
+
+  // All other DoNotRetryIOExceptions
+  static constexpr const char* kDoNotRetryIOException =
+      "org.apache.hadoop.hbase.DoNotRetryIOException";
+  static constexpr const char* kTableNotFoundException =
+      "org.apache.hadoop.hbase.TableNotFoundException";
+  static constexpr const char* kTableNotEnabledException =
+      "org.apache.hadoop.hbase.TableNotEnabledException";
+  static constexpr const char* kCoprocessorException =
+      "org.apache.hadoop.hbase.coprocessor.CoprocessorException";
+  static constexpr const char* kBypassCoprocessorException =
+      "org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException";
+  static constexpr const char* kInvalidFamilyOperationException =
+      "org.apache.hadoop.hbase.InvalidFamilyOperationException";
+  static constexpr const char* kServerTooBusyException =
+      "org.apache.hadoop.hbase.ipc.ServerTooBusyException";  // This should NOT be DNRIOE?
+  static constexpr const char* kFailedSanityCheckException =
+      "org.apache.hadoop.hbase.exceptions.FailedSanityCheckException";
+  static constexpr const char* kCorruptHFileException =
+      "org.apache.hadoop.hbase.io.hfile.CorruptHFileException";
+  static constexpr const char* kLabelAlreadyExistsException =
+      "org.apache.hadoop.hbase.security.visibility.LabelAlreadyExistsException";
+  static constexpr const char* kFatalConnectionException =
+      "org.apache.hadoop.hbase.ipc.FatalConnectionException";
+  static constexpr const char* kUnsupportedCryptoException =
+      "org.apache.hadoop.hbase.ipc.UnsupportedCryptoException";
+  static constexpr const char* kUnsupportedCellCodecException =
+      "org.apache.hadoop.hbase.ipc.UnsupportedCellCodecException";
+  static constexpr const char* kEmptyServiceNameException =
+      "org.apache.hadoop.hbase.ipc.EmptyServiceNameException";
+  static constexpr const char* kUnknownServiceException =
+      "org.apache.hadoop.hbase.ipc.UnknownServiceException";
+  static constexpr const char* kWrongVersionException =
+      "org.apache.hadoop.hbase.ipc.WrongVersionException";
+  static constexpr const char* kBadAuthException = "org.apache.hadoop.hbase.ipc.BadAuthException";
+  static constexpr const char* kUnsupportedCompressionCodecException =
+      "org.apache.hadoop.hbase.ipc.UnsupportedCompressionCodecException";
+  static constexpr const char* kDoNotRetryRegionException =
+      "org.apache.hadoop.hbase.client.DoNotRetryRegionException";
+  static constexpr const char* kRowTooBigException =
+      "org.apache.hadoop.hbase.client.RowTooBigException";
+  static constexpr const char* kRowTooBigExceptionDeprecated =
+      "org.apache.hadoop.hbase.regionserver.RowTooBigException";
+  static constexpr const char* kUnknownRegionException =
+      "org.apache.hadoop.hbase.UnknownRegionException";
+  static constexpr const char* kMergeRegionException =
+      "org.apache.hadoop.hbase.exceptions.MergeRegionException";
+  static constexpr const char* kNoServerForRegionException =
+      "org.apache.hadoop.hbase.client.NoServerForRegionException";
+  static constexpr const char* kQuotaExceededException =
+      "org.apache.hadoop.hbase.quotas.QuotaExceededException";
+  static constexpr const char* kSpaceLimitingException =
+      "org.apache.hadoop.hbase.quotas.SpaceLimitingException";
+  static constexpr const char* kThrottlingException =
+      "org.apache.hadoop.hbase.quotas.ThrottlingException";
+  static constexpr const char* kAccessDeniedException =
+      "org.apache.hadoop.hbase.security.AccessDeniedException";
+  static constexpr const char* kUnknownProtocolException =
+      "org.apache.hadoop.hbase.exceptions.UnknownProtocolException";
+  static constexpr const char* kRequestTooBigException =
+      "org.apache.hadoop.hbase.exceptions.RequestTooBigException";
+  static constexpr const char* kNotAllMetaRegionsOnlineException =
+      "org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException";
+  static constexpr const char* kConstraintException =
+      "org.apache.hadoop.hbase.constraint.ConstraintException";
+  static constexpr const char* kNoSuchColumnFamilyException =
+      "org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException";
+  static constexpr const char* kLeaseException =
+      "org.apache.hadoop.hbase.regionserver.LeaseException";
+  static constexpr const char* kInvalidLabelException =
+      "org.apache.hadoop.hbase.security.visibility.InvalidLabelException";
+
+  // TODO:
+  // These exceptions are not thrown in the regular read / write paths, although they are
+  // DoNotRetryIOExceptions. Add these to the list below in case we start doing Admin/DDL ops
+  // ReplicationPeerNotFoundException, XXXSnapshotException, NamespaceExistException,
+  // NamespaceNotFoundException, TableExistsException, TableNotDisabledException,
+  static const std::vector<const char*> kAllDoNotRetryIOExceptions;
+
+ public:
+  /**
+   * Returns whether or not the exception should be retried by looking at the
+   * client-side IOException, or RemoteException coming from server side.
+   */
+  static bool ShouldRetry(const folly::exception_wrapper& error);
+
+  /**
+   * Returns whether the java exception class extends DoNotRetryException.
+   * In the java side, we just have a hierarchy of Exception classes that we use
+   * both client side and server side. On the client side, we rethrow the server
+   * side exception by un-wrapping the exception from a RemoteException or a ServiceException
+   * (see ConnectionUtils.translateException() in Java).
+   * Since this object-hierarchy info is not available in C++ side, we are doing a
+   * very fragile catch-all list of all exception types in Java that extend the
+   * DoNotRetryException class type.
+   */
+  static bool IsJavaDoNotRetryException(const std::string& java_class_name);
+
+  /**
+   * Returns whether the scanner is closed when the client received the
+   * remote exception.
+   * Since the object-hierarchy info is not available in C++ side, we are doing a
+   * very fragile catch-all list of all exception types in Java that extend these
+   * three base classes: UnknownScannerException, NotServingRegionException,
+   * RegionServerStoppedException
+   */
+  static bool IsScannerClosed(const folly::exception_wrapper& exception);
+
+  static bool IsScannerOutOfOrder(const folly::exception_wrapper& exception);
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/security/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/security/BUCK b/hbase-native-client/include/hbase/security/BUCK
new file mode 100644
index 0000000..2e7530b
--- /dev/null
+++ b/hbase-native-client/include/hbase/security/BUCK
@@ -0,0 +1,30 @@
+##
+# 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.
+
+# This is the library dealing with a single connection
+# to a single server.
+cxx_library(
+    name="security",
+    header_namespace="hbase/security",
+    exported_headers=[
+        "user.h",
+    ],
+    deps=["//src/hbase/client:conf"],
+    compiler_flags=['-Weffc++'],
+    visibility=[
+        'PUBLIC',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/security/user.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/security/user.h b/hbase-native-client/include/hbase/security/user.h
new file mode 100644
index 0000000..c958d61
--- /dev/null
+++ b/hbase-native-client/include/hbase/security/user.h
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <glog/logging.h>
+#include <mutex>
+#include <string>
+#include "hbase/client/configuration.h"
+
+namespace hbase {
+namespace security {
+static constexpr const char* kKerberos = "kerberos";
+class User {
+ public:
+  explicit User(const std::string& user_name) : user_name_(user_name) {}
+  virtual ~User() = default;
+
+  std::string user_name() { return user_name_; }
+
+  static std::shared_ptr<User> defaultUser() { return std::make_shared<User>("__drwho"); }
+
+  static bool IsSecurityEnabled(const Configuration& conf) {
+    return conf.Get("hbase.security.authentication", "").compare(kKerberos) == 0;
+  }
+
+ private:
+  std::string user_name_;
+};
+}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/BUCK b/hbase-native-client/include/hbase/serde/BUCK
new file mode 100644
index 0000000..ee8ef82
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/BUCK
@@ -0,0 +1,37 @@
+##
+# 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.
+
+cxx_library(
+    name="serde",
+    header_namespace="hbase/serde",
+    exported_headers=[
+        "cell-scanner.h",
+        "cell-outputstream.h",
+        "codec.h",
+        "region-info.h",
+        "rpc-serde.h",
+        "server-name.h",
+        "table-name.h",
+        "zk.h",
+    ],
+    deps=[
+        "//src/hbase/if:if", "//third-party:folly", "//src/hbase/utils:utils", "//src/hbase/security:security"
+    ],
+    compiler_flags=['-Weffc++'],
+    visibility=[
+        'PUBLIC',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/cell-outputstream.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/cell-outputstream.h b/hbase-native-client/include/hbase/serde/cell-outputstream.h
new file mode 100644
index 0000000..963dd31
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/cell-outputstream.h
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+
+namespace hbase {
+
+class Cell;
+
+/**
+ * @brief Encoder / Decoder for Cells.
+ */
+class CellOutputStream {
+ public:
+  virtual ~CellOutputStream() {}
+
+  /**
+   * Implementation must copy the entire state of the Cell. If the written Cell is modified
+   * immediately after the write method returns, the modifications must have absolutely no effect
+   * on the copy of the Cell that was added in the write.
+   * @param cell Cell to write out
+   * @throws IOException
+   */
+  virtual void Write(const Cell& cell) = 0;
+
+  /**
+   * Let the implementation decide what to do.  Usually means writing accumulated data into a
+   * byte[] that can then be read from the implementation to be sent to disk, put in the block
+   * cache, or sent over the network.
+   * @throws IOException
+   */
+  virtual void Flush() = 0;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/cell-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/cell-scanner.h b/hbase-native-client/include/hbase/serde/cell-scanner.h
new file mode 100644
index 0000000..fe4a249
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/cell-scanner.h
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+#include <folly/io/IOBuf.h>
+#include <memory>
+
+namespace hbase {
+
+class Cell;
+
+/**
+ * @brief Interface for iterating over a sequence of Cells
+ */
+class CellScanner {
+ public:
+  virtual ~CellScanner() {}
+
+  /**
+   * @brief This method will be used to iterate the cells.
+   * Typical usage will be :-
+   * while(cell_scanner.Advance()){
+   *  auto current_cell = cell_scanner.Current();
+   * }
+   */
+  virtual bool Advance() = 0;
+
+  /**
+   * @brief returns the current cell
+   */
+  virtual const std::shared_ptr<Cell> Current() const = 0;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/codec.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/codec.h b/hbase-native-client/include/hbase/serde/codec.h
new file mode 100644
index 0000000..adc61b2
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/codec.h
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/io/IOBuf.h>
+#include <memory>
+
+#include "hbase/serde/cell-outputstream.h"
+#include "hbase/serde/cell-scanner.h"
+
+namespace hbase {
+
+/**
+ * @brief Encoder / Decoder for Cells.
+ */
+class Codec {
+ public:
+  virtual ~Codec() {}
+
+  class Encoder : public CellOutputStream {};
+
+  class Decoder : public CellScanner {};
+
+  virtual std::unique_ptr<Encoder> CreateEncoder() = 0;
+  virtual std::unique_ptr<Decoder> CreateDecoder(std::unique_ptr<folly::IOBuf> cell_block,
+                                                 uint32_t cell_block_start_offset,
+                                                 uint32_t cell_block_length) = 0;
+
+  /** @brief returns the java class name corresponding to this Codec implementation */
+  virtual const char* java_class_name() const = 0;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/region-info.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/region-info.h b/hbase-native-client/include/hbase/serde/region-info.h
new file mode 100644
index 0000000..075e092
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/region-info.h
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/Conv.h>
+#include <boost/algorithm/string/predicate.hpp>
+
+#include <string>
+
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+namespace pb {
+template <class String>
+void parseTo(String in, RegionInfo &out) {
+  // TODO(eclark): there has to be something better.
+  std::string s = folly::to<std::string>(in);
+
+  if (!boost::starts_with(s, "PBUF")) {
+    throw std::runtime_error("Region Info field doesn't contain preamble");
+  }
+  if (!out.ParseFromArray(s.data() + 4, s.size() - 4)) {
+    throw std::runtime_error("Bad protobuf for RegionInfo");
+  }
+}
+}  // namespace pb
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/rpc-serde.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/rpc-serde.h b/hbase-native-client/include/hbase/serde/rpc-serde.h
new file mode 100644
index 0000000..7c89347
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/rpc-serde.h
@@ -0,0 +1,155 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include <folly/ExceptionWrapper.h>
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/cell-scanner.h"
+#include "hbase/serde/codec.h"
+
+using namespace folly;
+// Forward
+namespace folly {
+class IOBuf;
+}
+namespace google {
+namespace protobuf {
+class Message;
+}
+}
+
+namespace hbase {
+
+/**
+ * @brief Class for serializing a deserializing rpc formatted data.
+ *
+ * RpcSerde is the one stop shop for reading/writing data to HBase daemons.
+ * It should throw exceptions if anything goes wrong.
+ */
+class RpcSerde {
+ public:
+  RpcSerde();
+  /**
+   * Constructor assumes the default auth type.
+   */
+  RpcSerde(std::shared_ptr<Codec> codec);
+
+  /**
+   * Destructor. This is provided just for testing purposes.
+   */
+  virtual ~RpcSerde() = default;
+
+  /**
+   * Pase a message in the delimited format.
+   *
+   * A message in delimited format consists of the following:
+   *
+   * - a protobuf var int32.
+   * - A protobuf object serialized.
+   */
+  int ParseDelimited(const folly::IOBuf *buf, google::protobuf::Message *msg);
+
+  /**
+   * Create a new connection preamble in a new IOBuf.
+   */
+  static std::unique_ptr<folly::IOBuf> Preamble(bool secure);
+
+  /**
+   * Create the header protobuf object and serialize it to a new IOBuf.
+   * Header is in the following format:
+   *
+   * - Big endian length
+   * - ConnectionHeader object serialized out.
+   */
+  std::unique_ptr<folly::IOBuf> Header(const std::string &user);
+
+  /**
+   * Take ownership of the passed buffer, and create a CellScanner using the
+   * Codec class to parse Cells out of the wire.
+   */
+  std::unique_ptr<CellScanner> CreateCellScanner(std::unique_ptr<folly::IOBuf> buf, uint32_t offset,
+                                                 uint32_t length);
+
+  /**
+   * Serialize a request message into a protobuf.
+   * Request consists of:
+   *
+   * - Big endian length
+   * - RequestHeader object
+   * - The passed in Message object
+   */
+  std::unique_ptr<folly::IOBuf> Request(const uint32_t call_id, const std::string &method,
+                                        const google::protobuf::Message *msg);
+
+  /**
+     * Serialize a response message into a protobuf.
+     * Request consists of:
+     *
+     * - Big endian length
+     * - ResponseHeader object
+     * - The passed in Message object
+     */
+  std::unique_ptr<folly::IOBuf> Response(const uint32_t call_id,
+                                         const google::protobuf::Message *msg);
+
+  /**
+   * Serialize a response message into a protobuf.
+   * Request consists of:
+   *
+   * - Big endian length
+   * - ResponseHeader object
+   * - The passed in hbase::Response object
+   */
+  std::unique_ptr<folly::IOBuf> Response(const uint32_t call_id,
+                                         const google::protobuf::Message *msg,
+                                         const folly::exception_wrapper &exception);
+
+  /**
+   * Serialize a message in the delimited format.
+   * Delimited format consists of the following:
+   *
+   * - A protobuf var int32
+   * - The message object seriailized after that.
+   */
+  std::unique_ptr<folly::IOBuf> SerializeDelimited(const google::protobuf::Message &msg);
+
+  /**
+   * Serilalize a message. This does not add any length prepend.
+   */
+  std::unique_ptr<folly::IOBuf> SerializeMessage(const google::protobuf::Message &msg);
+
+  /**
+   * Prepend a length IOBuf to the given IOBuf chain.
+   * This involves no copies or moves of the passed in data.
+   */
+  std::unique_ptr<folly::IOBuf> PrependLength(std::unique_ptr<folly::IOBuf> msg);
+
+ public:
+  static constexpr const char *HBASE_CLIENT_RPC_TEST_MODE = "hbase.client.rpc.test.mode";
+  static constexpr const bool DEFAULT_HBASE_CLIENT_RPC_TEST_MODE = false;
+
+ private:
+  /* data */
+  std::shared_ptr<Codec> codec_;
+  std::unique_ptr<pb::VersionInfo> CreateVersionInfo();
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/server-name.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/server-name.h b/hbase-native-client/include/hbase/serde/server-name.h
new file mode 100644
index 0000000..e650914
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/server-name.h
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/Conv.h>
+#include <folly/String.h>
+
+#include <string>
+
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+namespace pb {
+
+template <class String>
+void parseTo(String in, ServerName &out) {
+  // TODO see about getting rsplit into folly.
+  std::string s = folly::to<std::string>(in);
+
+  auto delim = s.rfind(":");
+  if (delim == std::string::npos) {
+    throw std::runtime_error("Couldn't parse server name");
+  }
+  out.set_host_name(s.substr(0, delim));
+  // Now keep everything after the : (delim + 1) to the end.
+  out.set_port(folly::to<int>(s.substr(delim + 1)));
+}
+
+}  // namespace pb
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/table-name.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/table-name.h b/hbase-native-client/include/hbase/serde/table-name.h
new file mode 100644
index 0000000..756f2aa
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/table-name.h
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Conv.h>
+#include <folly/String.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+namespace pb {
+
+// Provide folly::to<std::string>(TableName);
+template <class String>
+void toAppend(const TableName &in, String *result) {
+  if (!in.has_namespace_() || in.namespace_() == "default") {
+    folly::toAppend(in.qualifier(), result);
+  } else {
+    folly::toAppend(in.namespace_(), ':', in.qualifier(), result);
+  }
+}
+
+template <class String>
+void parseTo(String in, TableName &out) {
+  std::vector<std::string> v;
+  folly::split(":", in, v);
+
+  if (v.size() == 1) {
+    out.set_namespace_("default");
+    out.set_qualifier(v[0]);
+  } else {
+    out.set_namespace_(v[0]);
+    out.set_qualifier(v[1]);
+  }
+}
+
+}  // namespace pb
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/serde/zk.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/serde/zk.h b/hbase-native-client/include/hbase/serde/zk.h
new file mode 100644
index 0000000..5cadec2
--- /dev/null
+++ b/hbase-native-client/include/hbase/serde/zk.h
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+namespace google {
+namespace protobuf {
+class Message;
+}
+}
+namespace folly {
+class IOBuf;
+}
+
+namespace hbase {
+
+/** @brief A class to convert data from ZooKeeper to other formats.
+ *
+ * This class will convert data to and from Zookeeper into protobuf objects.
+ *
+ */
+class ZkDeserializer {
+ public:
+  /**
+   * Merge the data from a buffer into a given message.
+   *
+   * @param buf Naked pointer to iobuf containing data read from zookeeper.
+   * @param out Naked pointer into which the data will be merged. The message
+   * should be the correct type.
+   * @return returns true if the parsing was successful.
+   */
+  bool Parse(folly::IOBuf *buf, google::protobuf::Message *out);
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/BUCK b/hbase-native-client/include/hbase/utils/BUCK
new file mode 100644
index 0000000..d44e908
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/BUCK
@@ -0,0 +1,40 @@
+##
+# 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.
+
+cxx_library(
+    name="utils",
+    header_namespace="hbase/utils",
+    exported_headers=[
+        "bytes-util.h",
+        "connection-util.h",
+        "concurrent-map.h",
+        "optional.h",
+        "sys-util.h",
+        "time-util.h",
+        "user-util.h",
+        "version.h",
+    ],
+    deps=[
+        '//third-party:folly',
+    ],
+    tests=[":user-util-test"],
+    linker_flags=['-L/usr/local/lib', '-lkrb5'],
+    exported_linker_flags=['-L/usr/local/lib', '-lkrb5'],
+    visibility=[
+        'PUBLIC',
+    ],
+    compiler_flags=['-Weffc++'],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/bytes-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/bytes-util.h b/hbase-native-client/include/hbase/utils/bytes-util.h
new file mode 100644
index 0000000..6221bf0
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/bytes-util.h
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+
+namespace hbase {
+
+class BytesUtil {
+ private:
+  static const constexpr char kHexChars[] = "0123456789ABCDEF";
+
+ public:
+  static std::string ToStringBinary(const std::string& b) { return ToStringBinary(b, 0, b.size()); }
+  /**
+    * Write a printable representation of a byte array. Non-printable
+    * characters are hex escaped in the format \\x%02X, eg:
+    * \x00 \x05 etc
+    *
+    * @param b array to write out
+    * @param off offset to start at
+    * @param len length to write
+    * @return string output
+    */
+  static std::string ToStringBinary(const std::string& b, size_t off, size_t len);
+
+  static std::string ToString(int64_t value);
+
+  static int64_t ToInt64(std::string str);
+
+  static bool IsEmptyStartRow(const std::string& row) { return row == ""; }
+
+  static bool IsEmptyStopRow(const std::string& row) { return row == ""; }
+
+  static int32_t CompareTo(const std::string& a, const std::string& b) {
+    if (a < b) {
+      return -1;
+    }
+    if (a == b) {
+      return 0;
+    }
+    return 1;
+  }
+
+  /**
+   * Create the closest row after the specified row
+   */
+  static std::string CreateClosestRowAfter(std::string row) { return row.append(1, '\0'); }
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/concurrent-map.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/concurrent-map.h b/hbase-native-client/include/hbase/utils/concurrent-map.h
new file mode 100644
index 0000000..aebca0d
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/concurrent-map.h
@@ -0,0 +1,130 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <map>
+#include <memory>
+#include <shared_mutex>
+#include <unordered_map>
+#include <utility>
+
+namespace hbase {
+
+/**
+ * A concurrent version of std::unordered_map where we acquire a shared or exclusive
+ * lock for operations. This is NOT a highly-concurrent and scalable implementation
+ * since there is only one lock object.
+ * Replace this with tbb::concurrent_unordered_map or similar.
+ *
+ * Concurrency here is different than in Java. For example, the iterators returned from
+ * find() will not copy the key, value pairs.
+ */
+template <typename K, typename V>
+class concurrent_map {
+ public:
+  typedef K key_type;
+  typedef V mapped_type;
+  typedef std::pair<const key_type, mapped_type> value_type;
+  typedef typename std::unordered_map<K, V>::iterator iterator;
+  typedef typename std::unordered_map<K, V>::const_iterator const_iterator;
+
+  concurrent_map() : map_(), mutex_() {}
+  explicit concurrent_map(int32_t n) : map_(n), mutex_() {}
+
+  void insert(const value_type& value) {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    map_.insert(value);
+  }
+
+  /**
+   * Return the mapped object for this key. Be careful to not use the return reference
+   * to do assignment. I think it won't be thread safe
+   */
+  mapped_type& at(const key_type& key) {
+    std::shared_lock<std::shared_timed_mutex> lock(mutex_);
+    iterator where = map_.find(key);
+    if (where == end()) {
+      std::runtime_error("Key not found");
+    }
+    return where->second;
+  }
+
+  mapped_type& operator[](const key_type& key) {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    iterator where = map_.find(key);
+    if (where == end()) {
+      return map_[key];
+    }
+    return where->second;
+  }
+
+  /**
+   * Atomically finds the entry and removes it from the map, returning
+   * the previously associated value.
+   */
+  mapped_type find_and_erase(const K& key) {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    auto search = map_.find(key);
+    // It's an error if it's not there.
+    CHECK(search != end());
+    auto val = std::move(search->second);
+    map_.erase(key);
+    return val;
+  }
+
+  void erase(const K& key) {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    map_.erase(key);
+  }
+
+  iterator begin() { return map_.begin(); }
+
+  const_iterator begin() const { return map_.begin(); }
+
+  const_iterator cbegin() const { return map_.begin(); }
+
+  iterator end() { return map_.end(); }
+
+  const_iterator end() const { return map_.end(); }
+
+  const_iterator cend() const { return map_.end(); }
+
+  iterator find(const K& key) {
+    std::shared_lock<std::shared_timed_mutex> lock(mutex_);
+    return map_.find(key);
+  }
+
+  // TODO: find(), at() returning const_iterator
+
+  bool empty() const {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    return map_.empty();
+  }
+
+  void clear() {
+    std::unique_lock<std::shared_timed_mutex> lock(mutex_);
+    map_.clear();
+  }
+
+ private:
+  std::shared_timed_mutex mutex_;
+  std::unordered_map<K, V> map_;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/connection-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/connection-util.h b/hbase-native-client/include/hbase/utils/connection-util.h
new file mode 100644
index 0000000..b6a53e6
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/connection-util.h
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <algorithm>
+#include <climits>
+#include <cstdlib>
+#include <memory>
+#include <vector>
+#include "hbase/utils/time-util.h"
+
+namespace hbase {
+class ConnectionUtils {
+ public:
+  static int Retries2Attempts(const int& retries) {
+    return std::max(1, retries == INT_MAX ? INT_MAX : retries + 1);
+  }
+
+  /* Add a delta to avoid timeout immediately after a retry sleeping. */
+  static const uint64_t kSleepDeltaNs = 1000000;
+
+  static const std::vector<uint32_t> kRetryBackoff;
+  /**
+   * Calculate pause time. Built on {@link kRetryBackoff}.
+   * @param pause time to pause
+   * @param tries amount of tries
+   * @return How long to wait after <code>tries</code> retries
+   */
+  static int64_t GetPauseTime(const int64_t& pause, const int32_t& tries) {
+    int32_t ntries = tries;
+    if (static_cast<size_t>(ntries) >= kRetryBackoff.size()) {
+      ntries = kRetryBackoff.size() - 1;
+    }
+    if (ntries < 0) {
+      ntries = 0;
+    }
+
+    int64_t normal_pause = pause * kRetryBackoff[ntries];
+    // 1% possible jitter
+    float r = static_cast<float>(std::rand()) / static_cast<float>(RAND_MAX);
+    int64_t jitter = (int64_t)(normal_pause * r * 0.01f);
+    return normal_pause + jitter;
+  }
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/optional.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/optional.h b/hbase-native-client/include/hbase/utils/optional.h
new file mode 100644
index 0000000..a05eab5
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/optional.h
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <experimental/optional>
+
+namespace hbase {
+
+/**
+ * An optional value that may or may not be present.
+ */
+template <class T>
+using optional = std::experimental::optional<T>;
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/sys-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/sys-util.h b/hbase-native-client/include/hbase/utils/sys-util.h
new file mode 100644
index 0000000..68f00d7
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/sys-util.h
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <type_traits>
+
+namespace hbase {
+
+class SysUtil {
+ public:
+  template <class BASE, typename DERIVED>
+  static constexpr bool InstanceOf(const DERIVED& object) {
+    return !dynamic_cast<const BASE*>(&object);
+  }
+
+  template <typename BASE, typename DERIVED>
+  static constexpr bool InstanceOf() {
+    return std::is_base_of<BASE, DERIVED>();
+  }
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/time-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/time-util.h b/hbase-native-client/include/hbase/utils/time-util.h
new file mode 100644
index 0000000..165e9f1
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/time-util.h
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <chrono>
+#include <string>
+
+namespace hbase {
+
+class TimeUtil {
+ public:
+  static inline int64_t ToMillis(const int64_t& nanos) {
+    return std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::nanoseconds(nanos))
+        .count();
+  }
+
+  static inline std::chrono::milliseconds ToMillis(const std::chrono::nanoseconds& nanos) {
+    return std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::nanoseconds(nanos));
+  }
+
+  static inline std::chrono::nanoseconds ToNanos(const std::chrono::milliseconds& millis) {
+    return std::chrono::duration_cast<std::chrono::nanoseconds>(millis);
+  }
+
+  static inline std::chrono::nanoseconds MillisToNanos(const int64_t& millis) {
+    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::milliseconds(millis));
+  }
+
+  static inline std::chrono::nanoseconds SecondsToNanos(const int64_t& secs) {
+    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::seconds(secs));
+  }
+
+  static inline std::string ToMillisStr(const std::chrono::nanoseconds& nanos) {
+    return std::to_string(std::chrono::duration_cast<std::chrono::milliseconds>(nanos).count());
+  }
+
+  static inline int64_t GetNowNanos() {
+    auto duration = std::chrono::high_resolution_clock::now().time_since_epoch();
+    return std::chrono::duration_cast<std::chrono::nanoseconds>(duration).count();
+  }
+
+  static inline int64_t ElapsedMillis(const int64_t& start_ns) {
+    return std::chrono::duration_cast<std::chrono::milliseconds>(
+               std::chrono::nanoseconds(GetNowNanos() - start_ns))
+        .count();
+  }
+
+  static inline std::string ElapsedMillisStr(const int64_t& start_ns) {
+    return std::to_string(std::chrono::duration_cast<std::chrono::milliseconds>(
+                              std::chrono::nanoseconds(GetNowNanos() - start_ns))
+                              .count());
+  }
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/utils/user-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/utils/user-util.h b/hbase-native-client/include/hbase/utils/user-util.h
new file mode 100644
index 0000000..6258c85
--- /dev/null
+++ b/hbase-native-client/include/hbase/utils/user-util.h
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <mutex>
+#include <string>
+
+namespace hbase {
+
+/**
+ * @brief Class to help with user/group information.
+ *
+ * This class will get the current user, and information about them. It caches
+ * the user information after the first invocation.
+ */
+class UserUtil {
+ public:
+  /**
+   * Constructor.
+   */
+  UserUtil();
+
+  /**
+   * Get the username of the user owning this process. This is thread safe and
+   * lockless for every invocation other than the first one.
+   */
+  std::string user_name(bool secure = false);
+
+ private:
+  /**
+   * Compute the username. This will block.
+   */
+  void compute_user_name(bool secure);
+  std::once_flag once_flag_;
+  std::string user_name_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/security/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/security/BUCK b/hbase-native-client/security/BUCK
deleted file mode 100644
index f8a5695..0000000
--- a/hbase-native-client/security/BUCK
+++ /dev/null
@@ -1,30 +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.
-
-# This is the library dealing with a single connection
-# to a single server.
-cxx_library(
-    name="security",
-    exported_headers=[
-        "user.h",
-    ],
-    srcs=[],
-    deps=["//core:conf"],
-    compiler_flags=['-Weffc++'],
-    visibility=[
-        'PUBLIC',
-    ],)


[08/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/cell-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/cell-test.cc b/hbase-native-client/src/hbase/client/cell-test.cc
new file mode 100644
index 0000000..8a65aa5
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/cell-test.cc
@@ -0,0 +1,195 @@
+/*
+ * 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 "hbase/client/cell.h"
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <memory>
+
+using hbase::Cell;
+using hbase::CellType;
+
+TEST(CellTest, Constructor) {
+  std::string row = "row-value";
+  std::string family = "family-value";
+  std::string column = "column-value";
+  std::string value = "value-value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+
+  Cell cell{row, family, column, timestamp, value, cell_type};
+
+  EXPECT_EQ(row, cell.Row());
+  EXPECT_EQ(family, cell.Family());
+  EXPECT_EQ(column, cell.Qualifier());
+  EXPECT_EQ(value, cell.Value());
+  EXPECT_EQ(timestamp, cell.Timestamp());
+  EXPECT_EQ(cell_type, cell.Type());
+}
+
+TEST(CellTest, CopyConstructor) {
+  std::string row = "row-value";
+  std::string family = "family-value";
+  std::string column = "column-value";
+  std::string value = "value-value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+  Cell cell2{*cell};
+  cell = nullptr;
+
+  EXPECT_EQ(row, cell2.Row());
+  EXPECT_EQ(family, cell2.Family());
+  EXPECT_EQ(column, cell2.Qualifier());
+  EXPECT_EQ(value, cell2.Value());
+  EXPECT_EQ(timestamp, cell2.Timestamp());
+  EXPECT_EQ(cell_type, cell2.Type());
+}
+
+TEST(CellTest, CopyAssignment) {
+  std::string row = "row-value";
+  std::string family = "family-value";
+  std::string column = "column-value";
+  std::string value = "value-value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
+  Cell cell2 = *cell;
+  cell = nullptr;
+
+  EXPECT_EQ(row, cell2.Row());
+  EXPECT_EQ(family, cell2.Family());
+  EXPECT_EQ(column, cell2.Qualifier());
+  EXPECT_EQ(value, cell2.Value());
+  EXPECT_EQ(timestamp, cell2.Timestamp());
+  EXPECT_EQ(cell_type, cell2.Type());
+}
+
+TEST(CellTest, CellRowTest) {
+  std::string row = "only-row";
+  std::string family = "D";
+  std::string column = "";
+  std::string value = "";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+  Cell cell{row, family, column, timestamp, value, cell_type};
+
+  EXPECT_EQ(row, cell.Row());
+  EXPECT_EQ(family, cell.Family());
+  EXPECT_EQ(column, cell.Qualifier());
+  EXPECT_EQ(value, cell.Value());
+  EXPECT_EQ(timestamp, cell.Timestamp());
+  EXPECT_EQ(cell_type, cell.Type());
+}
+
+TEST(CellTest, CellRowFamilyTest) {
+  std::string row = "only-row";
+  std::string family = "only-family";
+  std::string column = "";
+  std::string value = "";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+  Cell cell{row, family, column, timestamp, value, cell_type};
+
+  EXPECT_EQ(row, cell.Row());
+  EXPECT_EQ(family, cell.Family());
+  EXPECT_EQ(column, cell.Qualifier());
+  EXPECT_EQ(value, cell.Value());
+  EXPECT_EQ(timestamp, cell.Timestamp());
+  EXPECT_EQ(cell_type, cell.Type());
+}
+
+TEST(CellTest, CellRowFamilyValueTest) {
+  std::string row = "only-row";
+  std::string family = "only-family";
+  std::string column = "";
+  std::string value = "only-value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+
+  Cell cell{row, family, column, timestamp, value, cell_type};
+
+  EXPECT_EQ(row, cell.Row());
+  EXPECT_EQ(family, cell.Family());
+  EXPECT_EQ(column, cell.Qualifier());
+  EXPECT_EQ(value, cell.Value());
+  EXPECT_EQ(timestamp, cell.Timestamp());
+  EXPECT_EQ(cell_type, cell.Type());
+}
+
+TEST(CellTest, CellRowFamilyColumnValueTest) {
+  std::string row = "only-row";
+  std::string family = "only-family";
+  std::string column = "only-column";
+  std::string value = "only-value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  CellType cell_type = CellType::PUT;
+  Cell cell{row, family, column, timestamp, value, cell_type};
+
+  EXPECT_EQ(row, cell.Row());
+  EXPECT_EQ(family, cell.Family());
+  EXPECT_EQ(column, cell.Qualifier());
+  EXPECT_EQ(value, cell.Value());
+  EXPECT_EQ(timestamp, cell.Timestamp());
+  EXPECT_EQ(cell_type, cell.Type());
+}
+
+TEST(CellTest, CellDebugString) {
+  CellType cell_type = CellType::PUT;
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  std::string value = "value";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+
+  Cell cell{row, family, column, timestamp, value, cell_type};
+  LOG(INFO) << cell.DebugString();
+  EXPECT_EQ("row/family:column/LATEST_TIMESTAMP/PUT/vlen=5/seqid=0", cell.DebugString());
+
+  Cell cell2{row, "", column, 42, value, CellType::DELETE};
+  LOG(INFO) << cell2.DebugString();
+  EXPECT_EQ("row/column/42/DELETE/vlen=5/seqid=0", cell2.DebugString());
+}
+
+TEST(CellTest, CellEstimatedSize) {
+  CellType cell_type = CellType::PUT;
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+
+  Cell empty{"a", "a", "", timestamp, "", cell_type};
+  Cell cell1{"aa", "a", "", timestamp, "", cell_type};
+  Cell cell2{"a", "aa", "", timestamp, "", cell_type};
+  Cell cell3{"a", "a", "a", timestamp, "", cell_type};
+  Cell cell4{"a", "a", "", timestamp, "a", cell_type};
+  Cell cell5{"a", "a", "", timestamp, "a", CellType::DELETE};
+  Cell cell6{"aaaaaa", "a", "", timestamp, "a", cell_type};
+
+  LOG(INFO) << empty.EstimatedSize();
+  LOG(INFO) << cell1.EstimatedSize();
+
+  EXPECT_TRUE(empty.EstimatedSize() > sizeof(Cell));
+  EXPECT_TRUE(cell1.EstimatedSize() > empty.EstimatedSize());
+  EXPECT_EQ(cell1.EstimatedSize(), cell2.EstimatedSize());
+  EXPECT_EQ(cell2.EstimatedSize(), cell3.EstimatedSize());
+  EXPECT_EQ(cell3.EstimatedSize(), cell4.EstimatedSize());
+  EXPECT_EQ(cell4.EstimatedSize(), cell5.EstimatedSize());
+  EXPECT_TRUE(cell6.EstimatedSize() > cell1.EstimatedSize());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/cell.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/cell.cc b/hbase-native-client/src/hbase/client/cell.cc
new file mode 100644
index 0000000..da2c11f
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/cell.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 "hbase/client/cell.h"
+#include <climits>
+#include <limits>
+#include <stdexcept>
+
+#include "folly/Conv.h"
+#include "hbase/utils/bytes-util.h"
+
+namespace hbase {
+
+Cell::Cell(const std::string &row, const std::string &family, const std::string &qualifier,
+           const int64_t timestamp, const std::string &value, const hbase::CellType &cell_type)
+    : row_(row),
+      family_(family),
+      qualifier_(qualifier),
+      timestamp_(timestamp),
+      cell_type_(cell_type),
+      value_(value),
+      sequence_id_(0) {
+  if (0 == row.size()) throw std::runtime_error("Row size should be greater than 0");
+
+  if (0 >= timestamp) throw std::runtime_error("Timestamp should be greater than 0");
+}
+
+Cell::Cell(const Cell &cell)
+    : row_(cell.row_),
+      family_(cell.family_),
+      qualifier_(cell.qualifier_),
+      timestamp_(cell.timestamp_),
+      cell_type_(cell.cell_type_),
+      value_(cell.value_),
+      sequence_id_(cell.sequence_id_) {}
+
+Cell &Cell::operator=(const Cell &cell) {
+  row_ = cell.row_;
+  family_ = cell.family_;
+  qualifier_ = cell.qualifier_;
+  timestamp_ = cell.timestamp_;
+  cell_type_ = cell.cell_type_;
+  value_ = cell.value_;
+  sequence_id_ = cell.sequence_id_;
+
+  return *this;
+}
+
+Cell::~Cell() {}
+
+const std::string &Cell::Row() const { return row_; }
+
+const std::string &Cell::Family() const { return family_; }
+
+const std::string &Cell::Qualifier() const { return qualifier_; }
+
+int64_t Cell::Timestamp() const { return timestamp_; }
+
+const std::string &Cell::Value() const { return value_; }
+
+hbase::CellType Cell::Type() const { return cell_type_; }
+
+int64_t Cell::SequenceId() const { return sequence_id_; }
+
+std::string Cell::DebugString() const {
+  std::string timestamp_str;
+  if (timestamp_ == std::numeric_limits<int64_t>::max()) {
+    timestamp_str = "LATEST_TIMESTAMP";
+  } else {
+    timestamp_str = folly::to<std::string>(timestamp_);
+  }
+
+  return BytesUtil::ToStringBinary(row_) + "/" + BytesUtil::ToStringBinary(family_) +
+         (family_.empty() ? "" : ":") + BytesUtil::ToStringBinary(qualifier_) + "/" +
+         timestamp_str + "/" + TypeToString(cell_type_) + "/vlen=" +
+         folly::to<std::string>(value_.size()) + "/seqid=" + folly::to<std::string>(sequence_id_);
+}
+
+const char *Cell::TypeToString(CellType type) {
+  switch (type) {
+    case CellType::MINIMUM:
+      return "MINIMUM";
+    case CellType::PUT:
+      return "PUT";
+    case CellType::DELETE:
+      return "DELETE";
+    case CellType::DELETE_COLUMN:
+      return "DELETE_COLUMN";
+    case CellType::DELETE_FAMILY:
+      return "DELETE_FAMILY";
+    case CellType::MAXIMUM:
+      return "MAXIMUM";
+    default:
+      return "UNKNOWN";
+  }
+}
+
+size_t Cell::EstimatedSize() const {
+  size_t s = sizeof(Cell);
+  s += row_.capacity();
+  s += family_.capacity();
+  s += qualifier_.capacity();
+  s += value_.capacity();
+  return s;
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/client-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/client-test.cc b/hbase-native-client/src/hbase/client/client-test.cc
new file mode 100644
index 0000000..7141047
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/client-test.cc
@@ -0,0 +1,697 @@
+/*
+ * 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 <gtest/gtest.h>
+
+#include "hbase/client/append.h"
+#include "hbase/client/cell.h"
+#include "hbase/client/client.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/delete.h"
+#include "hbase/client/get.h"
+#include "hbase/client/hbase-configuration-loader.h"
+#include "hbase/client/increment.h"
+#include "hbase/client/put.h"
+#include "hbase/client/result.h"
+#include "hbase/client/table.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/test-util/test-util.h"
+#include "hbase/utils/bytes-util.h"
+
+using hbase::Cell;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::RetriesExhaustedException;
+using hbase::Put;
+using hbase::Table;
+using hbase::TestUtil;
+using std::experimental::nullopt;
+
+class ClientTest : public ::testing::Test {
+ public:
+  static const constexpr char *kDefHBaseConfPath = "./build/test-data/client-test/conf/";
+  static const constexpr char *kHBaseDefaultXml = "hbase-default.xml";
+  static const constexpr char *kHBaseSiteXml = "hbase-site.xml";
+  static const constexpr char *kHBaseXmlData =
+      "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
+      "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
+      "Software Foundation (ASF) under one\n * or more contributor license "
+      "agreements.  See the NOTICE file\n * distributed with this work for "
+      "additional information\n * regarding copyright ownership.  The ASF "
+      "licenses this file\n * to you under the Apache License, Version 2.0 "
+      "(the\n * \"License\"); you may not use this file except in compliance\n * "
+      "with the License.  You may obtain a copy of the License at\n *\n *     "
+      "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
+      "applicable law or agreed to in writing, software\n * distributed under "
+      "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
+      "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
+      "for the specific language governing permissions and\n * limitations under "
+      "the License.\n "
+      "*/\n-->\n<configuration>\n\n</configuration>";
+
+  static void WriteDataToFile(const std::string &file, const std::string &xml_data) {
+    std::ofstream hbase_conf;
+    hbase_conf.open(file.c_str());
+    hbase_conf << xml_data;
+    hbase_conf.close();
+  }
+
+  static void CreateHBaseConf(const std::string &dir, const std::string &file,
+                              const std::string xml_data) {
+    // Remove temp file always
+    boost::filesystem::remove((dir + file).c_str());
+    boost::filesystem::create_directories(dir.c_str());
+    WriteDataToFile((dir + file), xml_data);
+  }
+
+  static void CreateHBaseConfWithEnv() {
+    // Creating Empty Config Files so that we dont get a Configuration exception @Client
+    CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseXmlData);
+    // the hbase-site.xml would be persisted by MiniCluster
+    setenv("HBASE_CONF", kDefHBaseConfPath, 1);
+  }
+  static std::unique_ptr<hbase::TestUtil> test_util;
+
+  static void SetUpTestCase() {
+    google::InstallFailureSignalHandler();
+    test_util = std::make_unique<hbase::TestUtil>();
+    test_util->StartMiniCluster(2);
+  }
+};
+std::unique_ptr<hbase::TestUtil> ClientTest::test_util = nullptr;
+
+TEST_F(ClientTest, EmptyConfigurationPassedToClient) { ASSERT_ANY_THROW(hbase::Client client); }
+
+TEST_F(ClientTest, ConfigurationPassedToClient) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  ClientTest::CreateHBaseConfWithEnv();
+
+  // Create Configuration
+  hbase::HBaseConfigurationLoader loader;
+  auto conf = loader.LoadDefaultResources();
+  // Create a client
+  hbase::Client client(conf.value());
+  client.Close();
+}
+
+TEST_F(ClientTest, DefaultConfiguration) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  ClientTest::CreateHBaseConfWithEnv();
+
+  // Create Configuration
+  hbase::Client client;
+  client.Close();
+}
+
+TEST_F(ClientTest, Append) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("t", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+  std::string val1 = "a";
+  auto result = table->Append(hbase::Append{row}.Add("d", "1", val1));
+
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ(row, result->Row());
+  EXPECT_EQ(val1, *(result->Value("d", "1")));
+
+  std::string val2 = "b";
+  result = table->Append(hbase::Append{row}.Add("d", "1", val2));
+
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ(row, result->Row());
+  EXPECT_EQ("ab", *(result->Value("d", "1")));
+}
+
+TEST_F(ClientTest, PutGetDelete) {
+  // Using TestUtil to populate test data
+  std::string tableName = "t1";
+  ClientTest::test_util->CreateTable(tableName, "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(tableName);
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  // Perform Puts
+  std::string valExtra = "value for extra";
+  std::string valExt = "value for ext";
+  table->Put(Put{row}.AddColumn("d", "1", "value1"));
+  // Put two values for column "extra"
+  table->Put(Put{row}.AddColumn("d", "extra", "1st val extra"));
+  usleep(1000);
+  table->Put(Put{row}.AddColumn("d", "extra", valExtra));
+  table->Put(Put{row}.AddColumn("d", "ext", valExt));
+
+  // Perform the Get
+  hbase::Get get(row);
+  auto result = table->Get(get);
+
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ("test1", result->Row());
+  EXPECT_EQ("value1", *(result->Value("d", "1")));
+  EXPECT_EQ(valExtra, *(result->Value("d", "extra")));
+  auto cell = *(result->ColumnCells("d", "extra"))[0];
+  auto tsExtra = cell.Timestamp();
+  auto tsExt = (*(result->ColumnCells("d", "ext"))[0]).Timestamp();
+
+  // delete column "1"
+  table->Delete(hbase::Delete{row}.AddColumn("d", "1"));
+  result = table->Get(get);
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
+  EXPECT_EQ(valExtra, *(result->Value("d", "extra")));
+
+  // delete cell from column "extra" with timestamp tsExtra
+  table->Delete(hbase::Delete{row}.AddColumn("d", "extra", tsExtra));
+  result = table->Get(get);
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
+  ASSERT_TRUE(result->Value("d", "extra") != nullopt) << "Column extra should have value";
+  EXPECT_EQ(valExt, *(result->Value("d", "ext"))) << "Column ext should have value";
+
+  // delete all cells from "extra" column
+  table->Delete(hbase::Delete{row}.AddColumns("d", "extra"));
+  result = table->Get(get);
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
+  ASSERT_FALSE(result->Value("d", "extra")) << "Column extra should be gone";
+  EXPECT_EQ(valExt, *(result->Value("d", "ext"))) << "Column ext should have value";
+
+  // Delete the row and verify that subsequent Get returns nothing
+  table->Delete(hbase::Delete{row}.AddFamily("d"));
+  result = table->Get(get);
+  ASSERT_TRUE(result->IsEmpty()) << "Result should be empty.";
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, Increment) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("t1", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t1");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+  int64_t incr1 = 1235;
+  auto result = table->Increment(hbase::Increment{row}.AddColumn("d", "1", incr1));
+  EXPECT_EQ(row, result->Row());
+
+  long l = hbase::BytesUtil::ToInt64(*(result->Value("d", "1")));
+  EXPECT_EQ(incr1, l);
+
+  int64_t incr2 = -2;
+  result = table->Increment(hbase::Increment{row}.AddColumn("d", "1", incr2));
+
+  EXPECT_EQ(row, result->Row());
+  EXPECT_EQ(incr1 + incr2, hbase::BytesUtil::ToInt64(*(result->Value("d", "1"))));
+}
+
+TEST_F(ClientTest, CheckAndPut) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("check", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("check");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  // Perform Puts
+  table->Put(Put{row}.AddColumn("d", "1", "value1"));
+  auto result = table->CheckAndPut(row, "d", "1", "value1", Put{row}.AddColumn("d", "1", "value2"));
+  ASSERT_TRUE(result) << "CheckAndPut didn't replace value";
+
+  result = table->CheckAndPut(row, "d", "1", "value1", Put{row}.AddColumn("d", "1", "value3"));
+
+  // Perform the Get
+  hbase::Get get(row);
+  auto result1 = table->Get(get);
+  EXPECT_EQ("value2", *(result1->Value("d", "1")));
+  ASSERT_FALSE(result) << "CheckAndPut shouldn't replace value";
+}
+
+TEST_F(ClientTest, CheckAndDelete) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("checkDel", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("checkDel");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  auto val1 = "value1";
+
+  // Perform Puts
+  table->Put(Put{row}.AddColumn("d", "1", val1));
+  table->Put(Put{row}.AddColumn("d", "2", "value2"));
+  auto result = table->CheckAndDelete(row, "d", "1", val1, hbase::Delete{row}.AddColumn("d", "2"));
+  ASSERT_TRUE(result) << "CheckAndDelete didn't replace value";
+
+  // Perform the Get
+  hbase::Get get(row);
+  auto result1 = table->Get(get);
+  EXPECT_EQ(val1, *(result1->Value("d", "1")));
+  ASSERT_FALSE(result1->Value("d", "2")) << "Column 2 should be gone";
+}
+
+TEST_F(ClientTest, PutGet) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("t", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  // Perform Puts
+  table->Put(Put{"test1"}.AddColumn("d", "1", "value1"));
+  table->Put(Put{"test1"}.AddColumn("d", "extra", "value for extra"));
+
+  // Perform the Get
+  hbase::Get get(row);
+  auto result = table->Get(get);
+
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ("test1", result->Row());
+  EXPECT_EQ("value1", *(result->Value("d", "1")));
+  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, GetForNonExistentTable) {
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t_not_exists");
+  auto row = "test1";
+
+  // Get to be performed on above HBase Table
+  hbase::Get get(row);
+
+  ClientTest::test_util->conf()->SetInt("hbase.client.retries.number", 5);
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  // Perform the Get
+  try {
+    table->Get(get);
+    FAIL() << "Should have thrown RetriesExhaustedException";
+  } catch (const RetriesExhaustedException &ex) {
+    ASSERT_EQ(0, ex.num_retries());
+  } catch (...) {
+    FAIL() << "Should have thrown RetriesExhaustedException";
+  }
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, GetForNonExistentRow) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("t_exists", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t_exists");
+  auto row = "row_not_exists";
+
+  // Get to be performed on above HBase Table
+  hbase::Get get(row);
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  // Perform the Get
+  auto result = table->Get(get);
+  ASSERT_TRUE(result->IsEmpty()) << "Result should  be empty.";
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, PutsWithTimestamp) {
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable("t_puts_with_timestamp", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t_puts_with_timestamp");
+  auto row = "test1";
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  int64_t ts = 42;
+  // Perform Puts
+  table->Put(Put{"test1"}.AddColumn("d", "1", ts, "value1"));
+  auto cell =
+      std::make_unique<Cell>("test1", "d", "extra", ts, "value for extra", hbase::CellType::PUT);
+  table->Put(Put{"test1"}.Add(std::move(cell)));
+
+  // Perform the Get
+  hbase::Get get(row);
+  auto result = table->Get(get);
+
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ("test1", result->Row());
+  EXPECT_EQ("value1", *(result->Value("d", "1")));
+  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
+  EXPECT_EQ(ts, result->ColumnLatestCell("d", "1")->Timestamp());
+  EXPECT_EQ(ts, result->ColumnLatestCell("d", "extra")->Timestamp());
+
+  table->Close();
+  client.Close();
+}
+
+void SetClientParams() {
+  ClientTest::test_util->conf()->SetInt("hbase.client.cpu.thread.pool.size", 6);
+  ClientTest::test_util->conf()->SetInt("hbase.client.operation.timeout", 600000);
+  ClientTest::test_util->conf()->SetInt("hbase.client.retries.number", 7);
+  ClientTest::test_util->conf()->SetInt("hbase.client.start.log.errors.counter", 1);
+}
+
+void PerformPuts(uint64_t num_rows, std::shared_ptr<hbase::Client> client,
+                 const std::string &table_name) {
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+  auto table = client->Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+  // Perform Puts
+  for (uint64_t i = 0; i < num_rows; i++) {
+    table->Put(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
+                                                         "value" + std::to_string(i)));
+  }
+}
+
+void MakeGets(uint64_t num_rows, const std::string &row_prefix, std::vector<hbase::Get> &gets) {
+  // Perform the Gets
+  for (uint64_t i = 0; i < num_rows; ++i) {
+    auto row = "test" + std::to_string(i);
+    hbase::Get get(row);
+    gets.push_back(get);
+  }
+  gets.push_back(hbase::Get("test2"));
+  gets.push_back(hbase::Get("testextra"));
+}
+
+void TestMultiResults(uint64_t num_rows, const std::vector<std::shared_ptr<hbase::Result>> &results,
+                      const std::vector<hbase::Get> &gets) {
+  // Test the values, should be same as in put executed on hbase shell
+  ASSERT_TRUE(!results.empty()) << "Result vector shouldn't be empty.";
+
+  uint32_t i = 0;
+  for (; i < num_rows; ++i) {
+    ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row()
+                                        << " must not be empty";
+    EXPECT_EQ("test" + std::to_string(i), results[i]->Row());
+    EXPECT_EQ("value" + std::to_string(i), results[i]->Value("d", std::to_string(i)).value());
+  }
+  // We are inserting test2 twice so the below test should pass
+  ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row() << " must not be empty";
+
+  ++i;
+  ASSERT_TRUE(results[i]->IsEmpty()) << "Result for Get " << gets[i].row() << " must be empty";
+}
+
+TEST_F(ClientTest, MultiGets) {
+  std::string table_name = "t";
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable(table_name, "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  uint64_t num_rows = 50000;
+  PerformPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  std::vector<hbase::Get> gets;
+  MakeGets(num_rows, "test", gets);
+
+  auto results = table->Get(gets);
+
+  TestMultiResults(num_rows, results, gets);
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, MultiGetsWithRegionSplits) {
+  // Using TestUtil to populate test data
+  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
+                                "test500", "test600", "test700", "test800", "test900"};
+  std::string table_name = "t";
+  ClientTest::test_util->CreateTable(table_name, "d", keys);
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  uint64_t num_rows = 50000;
+  PerformPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
+
+  std::vector<hbase::Get> gets;
+  MakeGets(num_rows, "test", gets);
+
+  auto results = table->Get(gets);
+
+  TestMultiResults(num_rows, results, gets);
+
+  table->Close();
+  client.Close();
+}
+
+void PerformMultiPuts(uint64_t num_rows, std::shared_ptr<hbase::Client> client,
+                      const std::string &table_name) {
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+  auto table = client->Table(tn);
+  ASSERT_TRUE(table) << "Unable to get connection to Table.";
+  std::vector<hbase::Put> puts;
+  // Perform Puts
+  for (uint64_t i = 0; i < num_rows; i++) {
+    puts.push_back(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
+                                                             "value" + std::to_string(i)));
+  }
+  table->Put(puts);
+}
+
+void PerformMultiPuts(std::vector<hbase::Put> &puts, std::shared_ptr<Table> table) {
+  table->Put(puts);
+}
+
+TEST_F(ClientTest, MultiGetsWithMultiPuts) {
+  std::string table_name = "t";
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable(table_name, "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  uint64_t num_rows = 50000;
+  PerformMultiPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table) << "Unable to get connection to Table.";
+
+  std::vector<hbase::Get> gets;
+  MakeGets(num_rows, "test", gets);
+
+  auto results = table->Get(gets);
+
+  TestMultiResults(num_rows, results, gets);
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, MultiGetsWithMultiPutsAndSplitRegions) {
+  // Using TestUtil to populate test data
+  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
+                                "test500", "test600", "test700", "test800", "test900"};
+  std::string table_name = "t";
+  ClientTest::test_util->CreateTable(table_name, "d", keys);
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+
+  uint64_t num_rows = 50000;
+  PerformMultiPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+  ASSERT_TRUE(table) << "Unable to get connection to Table.";
+
+  std::vector<hbase::Get> gets;
+  MakeGets(num_rows, "test", gets);
+
+  auto results = table->Get(gets);
+
+  TestMultiResults(num_rows, results, gets);
+
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, MultiPuts) {
+  std::string table_name = "t";
+  // Using TestUtil to populate test data
+  ClientTest::test_util->CreateTable(table_name, "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+  std::shared_ptr<Table> table = client.Table(tn);
+  ASSERT_TRUE(table) << "Unable to get connection to Table.";
+
+  uint64_t num_rows = 80000;
+  uint64_t batch_num_rows = 10000;
+  std::vector<hbase::Put> puts;
+  for (uint64_t i = 0; i < num_rows;) {
+    puts.clear();
+    // accumulate batch_num_rows at a time
+    for (uint64_t j = 0; j < batch_num_rows && i < num_rows; ++j) {
+      hbase::Put put("test" + std::to_string(i));
+      put.AddColumn("d", std::to_string(i), "value" + std::to_string(i));
+      puts.push_back(put);
+      i++;
+    }
+    PerformMultiPuts(puts, table);
+  }
+  table->Close();
+  client.Close();
+}
+
+TEST_F(ClientTest, MultiPutsWithRegionSplits) {
+  // Using TestUtil to populate test data
+  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
+                                "test500", "test600", "test700", "test800", "test900"};
+  std::string table_name = "t";
+  ClientTest::test_util->CreateTable(table_name, "d", keys);
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>(table_name);
+
+  SetClientParams();
+
+  // Create a client
+  hbase::Client client(*ClientTest::test_util->conf());
+  std::shared_ptr<Table> table = client.Table(tn);
+  ASSERT_TRUE(table) << "Unable to get connection to Table.";
+
+  uint64_t num_rows = 80000;
+  uint64_t batch_num_rows = 10000;
+  std::vector<hbase::Put> puts;
+  for (uint64_t i = 0; i < num_rows;) {
+    puts.clear();
+    // accumulate batch_num_rows at a time
+    for (uint64_t j = 0; j < batch_num_rows && i < num_rows; ++j) {
+      hbase::Put put("test" + std::to_string(i));
+      put.AddColumn("d", std::to_string(i), "value" + std::to_string(i));
+      puts.push_back(put);
+      i++;
+    }
+    PerformMultiPuts(puts, table);
+  }
+  table->Close();
+  client.Close();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/client.cc b/hbase-native-client/src/hbase/client/client.cc
new file mode 100644
index 0000000..2a63c03
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/client.cc
@@ -0,0 +1,55 @@
+/*
+ * 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 "hbase/client/client.h"
+
+#include <glog/logging.h>
+#include <chrono>
+#include <exception>
+#include <memory>
+#include <utility>
+
+using hbase::pb::TableName;
+
+namespace hbase {
+
+Client::Client() {
+  HBaseConfigurationLoader loader;
+  auto conf = loader.LoadDefaultResources();
+  if (!conf) {
+    LOG(ERROR) << "Unable to create default Configuration object. Either hbase-default.xml or "
+                  "hbase-site.xml is absent in the search path or problems in XML parsing";
+    throw std::runtime_error("Configuration object not present.");
+  }
+  Init(conf.value());
+}
+
+Client::Client(const Configuration &conf) { Init(conf); }
+
+void Client::Init(const Configuration &conf) {
+  auto conf_ = std::make_shared<Configuration>(conf);
+  async_connection_ = AsyncConnectionImpl::Create(conf_);
+}
+
+std::unique_ptr<Table> Client::Table(const TableName &table_name) {
+  return std::make_unique<hbase::Table>(table_name, async_connection_);
+}
+
+void Client::Close() { async_connection_->Close(); }
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/configuration-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/configuration-test.cc b/hbase-native-client/src/hbase/client/configuration-test.cc
new file mode 100644
index 0000000..4201018
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/configuration-test.cc
@@ -0,0 +1,119 @@
+/*
+ * 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 "hbase/client/configuration.h"
+#include <gtest/gtest.h>
+
+using hbase::Configuration;
+
+TEST(Configuration, SetGetBool) {
+  Configuration conf;
+
+  /* test true/false */
+  conf.SetBool("bool_key1", true);
+  EXPECT_EQ(true, conf.GetBool("bool_key1", false));
+  conf.SetBool("bool_key2", false);
+  EXPECT_EQ(false, conf.GetBool("bool_key2", true));
+
+  /* test 1/0 */
+  conf.SetBool("bool_key3", 1);
+  EXPECT_EQ(true, conf.GetBool("bool_key3", false));
+  conf.SetBool("bool_key4", 0);
+  EXPECT_EQ(false, conf.GetBool("bool_key4", true));
+
+  /* test non zero integer */
+  conf.SetBool("bool_key5", 5);
+  EXPECT_EQ(true, conf.GetBool("bool_key5", false));
+  conf.SetBool("bool_key6", -1);
+  EXPECT_EQ(true, conf.GetBool("bool_key5", false));
+
+  /* test non zero float */
+  conf.SetBool("bool_key7", 5.1);
+  EXPECT_EQ(true, conf.GetBool("bool_key7", false));
+  conf.SetBool("bool_key8", -1.2);
+  EXPECT_EQ(true, conf.GetBool("bool_key8", false));
+}
+
+TEST(Configuration, SetGetForBool) {
+  Configuration conf;
+
+  /* test true/false */
+  conf.Set("bool_key1", "true");
+  EXPECT_EQ(true, conf.GetBool("bool_key1", false));
+  conf.Set("bool_key2", "false");
+  EXPECT_EQ(false, conf.GetBool("bool_key2", true));
+
+  /* test 1/0 */
+  conf.Set("bool_key3", "1");
+  EXPECT_EQ(true, conf.GetBool("bool_key3", false));
+  conf.Set("bool_key4", "0");
+  EXPECT_EQ(false, conf.GetBool("bool_key4", true));
+
+  /* test non zero integer */
+  conf.Set("bool_key5", "5");
+  EXPECT_THROW(conf.GetBool("bool_key5", false), std::runtime_error);
+  conf.Set("bool_key6", "-1");
+  EXPECT_THROW(conf.GetBool("bool_key6", false), std::runtime_error);
+
+  /* test non zero float */
+  conf.Set("bool_key7", "5.1");
+  EXPECT_THROW(conf.GetBool("bool_key7", false), std::runtime_error);
+  conf.Set("bool_key8", "-1.2");
+  EXPECT_THROW(conf.GetBool("bool_key8", false), std::runtime_error);
+}
+
+TEST(Configuration, SetGet) {
+  Configuration conf;
+
+  EXPECT_EQ(conf.Get("foo", "default"), "default");
+  conf.Set("foo", "bar");
+  EXPECT_EQ(conf.Get("foo", "default"), "bar");
+}
+
+TEST(Configuration, SetGetInt) {
+  Configuration conf;
+
+  EXPECT_EQ(conf.GetInt("foo", 0), 0);
+  conf.SetInt("foo", 42);
+  EXPECT_EQ(conf.GetInt("foo", 0), 42);
+}
+
+TEST(Configuration, SetGetLong) {
+  Configuration conf;
+
+  EXPECT_EQ(conf.GetLong("foo", 0), 0);
+  conf.SetLong("foo", 42);
+  EXPECT_EQ(conf.GetLong("foo", 0), 42);
+}
+
+TEST(Configuration, SetGetDouble) {
+  Configuration conf;
+
+  EXPECT_EQ(conf.GetDouble("foo", 0), 0);
+  conf.SetDouble("foo", 42.0);
+  EXPECT_EQ(conf.GetDouble("foo", 0), 42.0);
+}
+
+TEST(Configuration, SetGetBoolBasic) {
+  Configuration conf;
+
+  EXPECT_EQ(conf.GetBool("foo", false), false);
+  conf.SetInt("foo", true);
+  EXPECT_EQ(conf.GetInt("foo", false), true);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/configuration.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/configuration.cc b/hbase-native-client/src/hbase/client/configuration.cc
new file mode 100644
index 0000000..d829edc
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/configuration.cc
@@ -0,0 +1,244 @@
+/*
+ * 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 "hbase/client/configuration.h"
+
+#include <memory>
+#include <stdexcept>
+#include <utility>
+
+#include <glog/logging.h>
+#include <boost/format.hpp>
+#include <boost/lexical_cast.hpp>
+
+namespace hbase {
+
+Configuration::Configuration() : hb_property_() {}
+
+Configuration::Configuration(ConfigMap &config_map) : hb_property_(std::move(config_map)) {}
+
+Configuration::~Configuration() {}
+
+size_t Configuration::IsSubVariable(const std::string &expr, std::string &sub_variable) const {
+  size_t start_pos = expr.find("${");
+  if (std::string::npos != start_pos) {
+    size_t pos_next = expr.find("}", start_pos + 1);
+    if (std::string::npos != pos_next) {
+      sub_variable = expr.substr(start_pos + 2, pos_next - (start_pos + 2));
+    }
+  }
+  return start_pos;
+}
+
+std::string Configuration::SubstituteVars(const std::string &expr) const {
+  if (0 == expr.size()) return expr;
+
+  std::string eval(expr);
+  std::string value_to_be_replaced("");
+  std::string var("");
+  for (int i = 0; i < kMaxSubsts; i++) {
+    var = "";
+    size_t start_pos = IsSubVariable(eval, var);
+    if (start_pos != std::string::npos) {
+      // We are blindly checking for environment property at first.
+      // If we don't get any value from GetEnv, check in hbase-site.xml.
+      value_to_be_replaced = GetEnv(var).value_or(GetProperty(var).value_or(""));
+
+      // we haven't found any value yet so we are returning eval
+      if (0 == value_to_be_replaced.size()) {
+        return eval;
+      }
+
+      // return original expression if there is a loop
+      if (value_to_be_replaced == expr) {
+        return expr;
+      }
+
+      eval.replace(start_pos, var.size() + 3, value_to_be_replaced);
+
+    } else {
+      // No further expansion required.
+      return eval;
+    }
+  }
+  // We reached here if the loop is exhausted
+  // If MAX_SUBSTS is exhausted, check if more variable substitution is reqd.
+  // If any-more substitutions are reqd, throw an error.
+  var = "";
+  if (IsSubVariable(eval, var) != std::string::npos) {
+    throw std::runtime_error("Variable substitution depth too large: " +
+                             std::to_string(kMaxSubsts) + " " + expr);
+  } else {
+    return eval;
+  }
+}
+
+optional<std::string> Configuration::GetEnv(const std::string &key) const {
+  char buf[2048];
+
+  if ("user.name" == key) {
+#ifdef HAVE_GETLOGIN
+    return std::experimental::make_optional(getlogin());
+#else
+    DLOG(WARNING) << "Client user.name not implemented";
+    return optional<std::string>();
+#endif
+  }
+
+  if ("user.dir" == key) {
+#ifdef HAVE_GETCWD
+    if (getcwd(buf, sizeof(buf))) {
+      return std::experimental::make_optional(buf);
+    } else {
+      return optional<std::string>();
+    }
+#else
+    DLOG(WARNING) << "Client user.dir not implemented";
+    return optional<std::string>();
+#endif
+  }
+
+  if ("user.home" == key) {
+#if defined(HAVE_GETUID) && defined(HAVE_GETPWUID_R)
+    uid = getuid();
+    if (!getpwuid_r(uid, &pw, buf, sizeof(buf), &pwp)) {
+      return std::experimental::make_optional(buf);
+    } else {
+      return optional<std::string>();
+    }
+#else
+    DLOG(WARNING) << "Client user.home not implemented";
+    return optional<std::string>();
+#endif
+  }
+  return optional<std::string>();
+}
+
+optional<std::string> Configuration::GetProperty(const std::string &key) const {
+  auto found = hb_property_.find(key);
+  if (found != hb_property_.end()) {
+    return std::experimental::make_optional(found->second.value);
+  } else {
+    return optional<std::string>();
+  }
+}
+
+optional<std::string> Configuration::Get(const std::string &key) const {
+  optional<std::string> raw = GetProperty(key);
+  if (raw) {
+    return std::experimental::make_optional(SubstituteVars(*raw));
+  } else {
+    return optional<std::string>();
+  }
+}
+
+std::string Configuration::Get(const std::string &key, const std::string &default_value) const {
+  return Get(key).value_or(default_value);
+}
+
+optional<int32_t> Configuration::GetInt(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(boost::lexical_cast<int32_t>(*raw));
+    } catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<int32_t>();
+}
+
+int32_t Configuration::GetInt(const std::string &key, int32_t default_value) const {
+  return GetInt(key).value_or(default_value);
+}
+
+optional<int64_t> Configuration::GetLong(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(boost::lexical_cast<int64_t>(*raw));
+    } catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<int64_t>();
+}
+
+int64_t Configuration::GetLong(const std::string &key, int64_t default_value) const {
+  return GetLong(key).value_or(default_value);
+}
+
+optional<double> Configuration::GetDouble(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(boost::lexical_cast<double>(*raw));
+    } catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<double>();
+}
+
+double Configuration::GetDouble(const std::string &key, double default_value) const {
+  return GetDouble(key).value_or(default_value);
+}
+
+optional<bool> Configuration::GetBool(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    if (!strcasecmp((*raw).c_str(), "true") || !strcasecmp((*raw).c_str(), "1")) {
+      return std::experimental::make_optional(true);
+    } else if (!strcasecmp((*raw).c_str(), "false") || !strcasecmp((*raw).c_str(), "0")) {
+      return std::experimental::make_optional(false);
+    } else {
+      boost::format what("Unexpected value \"%s\" found being converted to bool for key \"%s\"");
+      what % (*raw);
+      what % key;
+      throw std::runtime_error(what.str());
+    }
+  }
+  return optional<bool>();
+}
+
+bool Configuration::GetBool(const std::string &key, bool default_value) const {
+  return GetBool(key).value_or(default_value);
+}
+
+void Configuration::Set(const std::string &key, const std::string &value) {
+  hb_property_[key] = value;
+}
+
+void Configuration::SetInt(const std::string &key, int32_t value) {
+  Set(key, boost::lexical_cast<std::string>(value));
+}
+
+void Configuration::SetLong(const std::string &key, int64_t value) {
+  Set(key, boost::lexical_cast<std::string>(value));
+}
+
+void Configuration::SetDouble(const std::string &key, double value) {
+  Set(key, boost::lexical_cast<std::string>(value));
+}
+
+void Configuration::SetBool(const std::string &key, bool value) {
+  Set(key, boost::lexical_cast<std::string>(value));
+}
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/delete-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/delete-test.cc b/hbase-native-client/src/hbase/client/delete-test.cc
new file mode 100644
index 0000000..4af0c40
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/delete-test.cc
@@ -0,0 +1,124 @@
+/*
+ * 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 <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "hbase/client/delete.h"
+#include "hbase/client/mutation.h"
+#include "hbase/utils/time-util.h"
+
+using hbase::Delete;
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Mutation;
+using hbase::TimeUtil;
+
+const constexpr int64_t Mutation::kLatestTimestamp;
+
+TEST(Delete, Row) {
+  Delete del{"foo"};
+  EXPECT_EQ("foo", del.row());
+}
+
+TEST(Delete, Timestamp) {
+  Delete del{"row"};
+
+  // test default timestamp
+  EXPECT_EQ(Mutation::kLatestTimestamp, del.TimeStamp());
+
+  // set custom timestamp
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  del.SetTimeStamp(ts);
+  EXPECT_EQ(ts, del.TimeStamp());
+
+  // Add a column with custom timestamp
+  del.AddColumn("f", "q");
+  auto &cell = del.FamilyMap().at("f")[0];
+  EXPECT_EQ(ts, cell->Timestamp());
+}
+
+TEST(Delete, HasFamilies) {
+  Delete del{"row"};
+
+  EXPECT_EQ(false, del.HasFamilies());
+
+  del.AddColumn("f", "q");
+  EXPECT_EQ(true, del.HasFamilies());
+}
+
+TEST(Delete, Add) {
+  CellType cell_type = CellType::DELETE;
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+  int64_t timestamp = std::numeric_limits<int64_t>::max();
+  auto cell = std::make_unique<Cell>(row, family, column, timestamp, "", cell_type);
+
+  // add first cell
+  Delete del{"row"};
+  del.Add(std::move(cell));
+  EXPECT_EQ(1, del.FamilyMap().size());
+  EXPECT_EQ(1, del.FamilyMap().at(family).size());
+
+  // add a non-matching row
+  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, "", cell_type);
+  Delete del2{"foo"};
+  ASSERT_THROW(del2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
+
+  // add a second cell with same family
+  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, "", cell_type);
+  del.Add(std::move(cell3));
+  EXPECT_EQ(1, del.FamilyMap().size());
+  EXPECT_EQ(2, del.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, "", cell_type);
+  del.Add(std::move(cell4));
+  EXPECT_EQ(2, del.FamilyMap().size());
+  EXPECT_EQ(1, del.FamilyMap().at("family-2").size());
+}
+
+TEST(Delete, AddColumn) {
+  std::string row = "row";
+  std::string family = "family";
+  std::string column = "column";
+
+  Delete del{"row"};
+  del.AddColumn(family, column);
+  EXPECT_EQ(1, del.FamilyMap().size());
+  EXPECT_EQ(1, del.FamilyMap().at(family).size());
+
+  // add a second cell with same family
+  del.AddColumn(family, "column-2");
+  EXPECT_EQ(1, del.FamilyMap().size());
+  EXPECT_EQ(2, del.FamilyMap().at(family).size());
+
+  // add a cell to a different family
+  del.AddColumn("family-2", column);
+  EXPECT_EQ(2, del.FamilyMap().size());
+  EXPECT_EQ(1, del.FamilyMap().at("family-2").size());
+
+  // use the AddColumn overload
+  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
+  del.AddColumn(family, column, ts);
+  EXPECT_EQ(2, del.FamilyMap().size());
+  EXPECT_EQ(3, del.FamilyMap().at(family).size());
+  auto &cell = del.FamilyMap().at(family)[2];
+  EXPECT_EQ(ts, cell->Timestamp());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/delete.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/delete.cc b/hbase-native-client/src/hbase/client/delete.cc
new file mode 100644
index 0000000..eed096d
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/delete.cc
@@ -0,0 +1,131 @@
+
+
+/*
+ * 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 "hbase/client/delete.h"
+#include <folly/Conv.h>
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+#include <utility>
+
+namespace hbase {
+
+/**
+ * @brief Add the specified column to this Delete operation.
+ * This is an expensive call in that on the server-side, it first does a
+ * get to find the latest versions timestamp.  Then it adds a delete using
+ * the fetched cells timestamp.
+ *  @param family family name
+ *  @param qualifier column qualifier
+ */
+Delete& Delete::AddColumn(const std::string& family, const std::string& qualifier) {
+  return AddColumn(family, qualifier, timestamp_);
+}
+
+/**
+ *  @brief Add the specified column to this Delete operation.
+ *  @param family family name
+ *  @param qualifier column qualifier
+ *  @param timestamp version timestamp
+ */
+Delete& Delete::AddColumn(const std::string& family, const std::string& qualifier,
+                          int64_t timestamp) {
+  if (timestamp < 0) {
+    throw std::runtime_error("Timestamp cannot be negative. ts=" +
+                             folly::to<std::string>(timestamp));
+  }
+
+  return Add(
+      std::make_unique<Cell>(row_, family, qualifier, timestamp, "", hbase::CellType::DELETE));
+}
+/**
+ * Delete all versions of the specified column.
+ * @param family family name
+ * @param qualifier column qualifier
+ */
+Delete& Delete::AddColumns(const std::string& family, const std::string& qualifier) {
+  return AddColumns(family, qualifier, timestamp_);
+}
+/**
+ * Delete all versions of the specified column with a timestamp less than
+ * or equal to the specified timestamp.
+ * @param family family name
+ * @param qualifier column qualifier
+ * @param timestamp maximum version timestamp
+ */
+Delete& Delete::AddColumns(const std::string& family, const std::string& qualifier,
+                           int64_t timestamp) {
+  if (timestamp < 0) {
+    throw std::runtime_error("Timestamp cannot be negative. ts=" +
+                             folly::to<std::string>(timestamp));
+  }
+
+  return Add(std::make_unique<Cell>(row_, family, qualifier, timestamp, "",
+                                    hbase::CellType::DELETE_COLUMN));
+}
+/**
+ * Delete all versions of all columns of the specified family.
+ * <p>
+ * Overrides previous calls to deleteColumn and deleteColumns for the
+ * specified family.
+ * @param family family name
+ */
+Delete& Delete::AddFamily(const std::string& family) { return AddFamily(family, timestamp_); }
+
+/**
+ * Delete all columns of the specified family with a timestamp less than
+ * or equal to the specified timestamp.
+ * <p>
+ * Overrides previous calls to deleteColumn and deleteColumns for the
+ * specified family.
+ * @param family family name
+ * @param timestamp maximum version timestamp
+ */
+Delete& Delete::AddFamily(const std::string& family, int64_t timestamp) {
+  const auto& it = family_map_.find(family);
+  if (family_map_.end() != it) {
+    it->second.clear();
+  } else {
+    family_map_[family];
+  }
+  return Add(
+      std::make_unique<Cell>(row_, family, "", timestamp, "", hbase::CellType::DELETE_FAMILY));
+}
+/**
+ * Delete all columns of the specified family with a timestamp equal to
+ * the specified timestamp.
+ * @param family family name
+ * @param timestamp version timestamp
+ */
+Delete& Delete::AddFamilyVersion(const std::string& family, int64_t timestamp) {
+  return Add(std::make_unique<Cell>(row_, family, "", timestamp, "",
+                                    hbase::CellType::DELETE_FAMILY_VERSION));
+}
+Delete& Delete::Add(std::unique_ptr<Cell> cell) {
+  if (cell->Row() != row_) {
+    throw std::runtime_error("The row in " + cell->DebugString() +
+                             " doesn't match the original one " + row_);
+  }
+
+  family_map_[cell->Family()].push_back(std::move(cell));
+  return *this;
+}
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/filter-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/filter-test.cc b/hbase-native-client/src/hbase/client/filter-test.cc
new file mode 100644
index 0000000..8a23913
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/filter-test.cc
@@ -0,0 +1,141 @@
+/*
+ * 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 <gtest/gtest.h>
+#include "hbase/client/client.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/get.h"
+#include "hbase/client/put.h"
+#include "hbase/client/result.h"
+#include "hbase/client/table.h"
+#include "hbase/if/Comparator.pb.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/table-name.h"
+#include "hbase/test-util/test-util.h"
+
+using hbase::Configuration;
+using hbase::Get;
+using hbase::Put;
+using hbase::FilterFactory;
+using hbase::Table;
+using hbase::TestUtil;
+using hbase::pb::CompareType;
+using hbase::ComparatorFactory;
+using hbase::Comparator;
+
+class FilterTest : public ::testing::Test {
+ protected:
+  static void SetUpTestCase() {
+    test_util_ = std::make_unique<TestUtil>();
+    test_util_->StartMiniCluster(2);
+  }
+
+  static void TearDownTestCase() { test_util_.release(); }
+
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+
+  static std::unique_ptr<TestUtil> test_util_;
+};
+
+std::unique_ptr<TestUtil> FilterTest::test_util_ = nullptr;
+
+TEST_F(FilterTest, GetWithColumnPrefixFilter) {
+  // write row1 with 3 columns (column_1, column_2, and foo_column)
+  FilterTest::test_util_->CreateTable("t", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t");
+  auto row = "row1";
+
+  // Gets to be performed on above HBase Table
+  Get get_all(row);  // expected to return all 3 columns
+  Get get_one(row);  // expected to return 1 column
+  Get get_two(row);  // expected to return 2 column
+
+  get_one.SetFilter(FilterFactory::ColumnPrefixFilter("foo_"));
+  get_two.SetFilter(FilterFactory::ColumnPrefixFilter("column_"));
+
+  // Create a client
+  hbase::Client client(*(FilterTest::test_util_->conf()));
+  auto table = client.Table(tn);
+
+  table->Put(Put{"row1"}.AddColumn("d", "column_1", "value1"));
+  table->Put(Put{"row1"}.AddColumn("d", "column_2", "value2"));
+  table->Put(Put{"row1"}.AddColumn("d", "foo_column", "value3"));
+
+  // Perform the Get
+  auto result_all = table->Get(get_all);
+  auto result_one = table->Get(get_one);
+  auto result_two = table->Get(get_two);
+
+  table->Close();
+  client.Close();
+
+  // Test the values
+  ASSERT_TRUE(!result_one->IsEmpty()) << "Result shouldn't be empty.";
+  ASSERT_TRUE(!result_two->IsEmpty()) << "Result shouldn't be empty.";
+  ASSERT_TRUE(!result_all->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ(row, result_one->Row());
+  EXPECT_EQ(row, result_two->Row());
+  EXPECT_EQ(row, result_all->Row());
+  EXPECT_EQ(1, result_one->Size());
+  EXPECT_EQ(2, result_two->Size());
+  EXPECT_EQ(3, result_all->Size());
+  EXPECT_EQ("value3", *(result_one->Value("d", "foo_column")));
+  EXPECT_EQ("value1", *(result_two->Value("d", "column_1")));
+  EXPECT_EQ("value2", *(result_two->Value("d", "column_2")));
+}
+
+TEST_F(FilterTest, GetWithQualifierFilter) {
+  // write row1 with 3 columns (a,b,c)
+  FilterTest::test_util_->CreateTable("t1", "d");
+
+  // Create TableName and Row to be fetched from HBase
+  auto tn = folly::to<hbase::pb::TableName>("t1");
+  auto row = "row1";
+
+  // Gets to be performed on above HBase Table
+  Get get(row);
+  get.SetFilter(FilterFactory::QualifierFilter(CompareType::GREATER_OR_EQUAL,
+                                               *ComparatorFactory::BinaryComparator("b")));
+
+  // Create a client
+  hbase::Client client(*(FilterTest::test_util_->conf()));
+
+  // Get connection to HBase Table
+  auto table = client.Table(tn);
+
+  table->Put(Put{"row1"}.AddColumn("d", "a", "value1"));
+  table->Put(Put{"row1"}.AddColumn("d", "b", "value2"));
+  table->Put(Put{"row1"}.AddColumn("d", "c", "value3"));
+
+  // Perform the Get
+  auto result = table->Get(get);
+
+  table->Close();
+  client.Close();
+
+  // Test the values
+  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
+  EXPECT_EQ(row, result->Row());
+  EXPECT_EQ(2, result->Size());
+  EXPECT_EQ("value2", *(result->Value("d", "b")));
+  EXPECT_EQ("value3", *(result->Value("d", "c")));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/get-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/get-test.cc b/hbase-native-client/src/hbase/client/get-test.cc
new file mode 100644
index 0000000..676d798
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/get-test.cc
@@ -0,0 +1,221 @@
+/*
+ * 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 "hbase/client/get.h"
+#include "hbase/client/cell.h"
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+using hbase::Cell;
+using hbase::Get;
+
+const int NUMBER_OF_GETS = 5;
+
+void CheckFamilies(Get &get) {
+  EXPECT_EQ(false, get.HasFamilies());
+  get.AddFamily("family-1");
+  EXPECT_EQ(true, get.HasFamilies());
+  EXPECT_EQ(1, get.FamilyMap().size());
+  for (const auto &family : get.FamilyMap()) {
+    EXPECT_STREQ("family-1", family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+  }
+  // Not allowed to add the same CF.
+  get.AddFamily("family-1");
+  EXPECT_EQ(1, get.FamilyMap().size());
+  get.AddFamily("family-2");
+  EXPECT_EQ(2, get.FamilyMap().size());
+  get.AddFamily("family-3");
+  EXPECT_EQ(3, get.FamilyMap().size());
+  int i = 1;
+  for (const auto &family : get.FamilyMap()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+    i += 1;
+  }
+
+  get.AddColumn("family-1", "column-1");
+  get.AddColumn("family-1", "column-2");
+  get.AddColumn("family-1", "");
+  get.AddColumn("family-1", "column-3");
+  get.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, get.FamilyMap().size());
+  auto it = get.FamilyMap().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, get.FamilyMap().end());
+}
+
+void CheckFamiliesAfterCopy(Get &get) {
+  EXPECT_EQ(true, get.HasFamilies());
+  EXPECT_EQ(3, get.FamilyMap().size());
+  int i = 1;
+  for (const auto &family : get.FamilyMap()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    i += 1;
+  }
+  // Check if the alreaday added CF's and CQ's are as expected
+  auto it = get.FamilyMap().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, get.FamilyMap().end());
+}
+
+void GetMethods(Get &get, const std::string &row) {
+  EXPECT_EQ(row, get.row());
+
+  CheckFamilies(get);
+  EXPECT_EQ(true, get.CacheBlocks());
+  get.SetCacheBlocks(false);
+  EXPECT_EQ(false, get.CacheBlocks());
+
+  EXPECT_EQ(hbase::pb::Consistency::STRONG, get.Consistency());
+  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, get.Consistency());
+
+  EXPECT_EQ(1, get.MaxVersions());
+  get.SetMaxVersions(2);
+  EXPECT_EQ(2, get.MaxVersions());
+  get.SetMaxVersions();
+  EXPECT_EQ(1, get.MaxVersions());
+
+  // Test initial values
+  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(std::numeric_limits<int64_t>::max(), get.Timerange().MaxTimeStamp());
+
+  // Set & Test new values using TimeRange and TimeStamp
+  get.SetTimeRange(1000, 2000);
+  EXPECT_EQ(1000, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(2000, get.Timerange().MaxTimeStamp());
+  get.SetTimeStamp(0);
+  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(1, get.Timerange().MaxTimeStamp());
+
+  // Test some exceptions
+  ASSERT_THROW(get.SetTimeRange(-1000, 2000), std::runtime_error);
+  ASSERT_THROW(get.SetTimeRange(1000, -2000), std::runtime_error);
+  ASSERT_THROW(get.SetTimeRange(1000, 200), std::runtime_error);
+  ASSERT_THROW(get.SetTimeStamp(std::numeric_limits<int64_t>::max()), std::runtime_error);
+
+  // Test some exceptions
+  ASSERT_THROW(get.SetMaxVersions(0), std::runtime_error);
+  ASSERT_THROW(get.SetMaxVersions(std::numeric_limits<uint32_t>::max() + 1), std::runtime_error);
+}
+
+TEST(Get, SingleGet) {
+  std::string row_str = "row-test";
+  Get get(row_str);
+  GetMethods(get, row_str);
+
+  Get get_tmp(row_str);
+  Get getcp(get_tmp);
+  GetMethods(getcp, row_str);
+
+  Get geteq("test");
+  geteq = get_tmp;
+  GetMethods(geteq, row_str);
+
+  // Adding the below tests as there were some concerns raised that the same
+  // vector of qualifiers in FamilyMap is being shared between copied objects
+  // Verify the source object's family map size before using it to copy.
+  EXPECT_EQ(3, get.FamilyMap().size());
+
+  Get getcp_fam(get);
+  // address of family maps should be different.
+  EXPECT_NE(&(get.FamilyMap()), &(getcp_fam.FamilyMap()));
+
+  // Add family to the source object
+  get.AddColumn("family-4", "column-A");
+  get.AddColumn("family-4", "column-B");
+  // Verify the source object's family map size
+  EXPECT_EQ(4, get.FamilyMap().size());
+  // Verify the source object's family elements
+  auto it = get.FamilyMap().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_STREQ("family-4", it->first.c_str());
+  EXPECT_EQ(2, it->second.size());
+  EXPECT_STREQ("column-A", it->second[0].c_str());
+  EXPECT_STREQ("column-B", it->second[1].c_str());
+  ++it;
+  EXPECT_EQ(it, get.FamilyMap().end());
+
+  // Verifying the copied object's families. It will remain unchanged and below
+  // tests should pass
+  CheckFamiliesAfterCopy(getcp_fam);
+}
+
+TEST(Get, MultiGet) {
+  std::vector<std::unique_ptr<Get>> gets;
+  for (int i = 0; i < NUMBER_OF_GETS; i++) {
+    std::string row_str = "row-test";
+    row_str += std::to_string(i);
+    auto get = std::make_unique<Get>(row_str);
+
+    GetMethods(*get, row_str);
+    gets.push_back(std::move(get));
+  }
+  EXPECT_EQ(NUMBER_OF_GETS, gets.size());
+}
+
+TEST(Get, Exception) {
+  std::string row(std::numeric_limits<int16_t>::max() + 1, 'X');
+  ASSERT_THROW(Get tmp = Get(row), std::runtime_error);
+  ASSERT_THROW(Get tmp = Get(""), std::runtime_error);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/get.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/get.cc b/hbase-native-client/src/hbase/client/get.cc
new file mode 100644
index 0000000..ba18a58
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/get.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 "hbase/client/get.h"
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+
+Get::~Get() {}
+
+Get::Get(const std::string &row) : Row(row) {}
+
+Get::Get(const Get &get) : Query(get) {
+  row_ = get.row_;
+  max_versions_ = get.max_versions_;
+  cache_blocks_ = get.cache_blocks_;
+  check_existence_only_ = get.check_existence_only_;
+  consistency_ = get.consistency_;
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
+}
+
+Get &Get::operator=(const Get &get) {
+  Query::operator=(get);
+  row_ = get.row_;
+  max_versions_ = get.max_versions_;
+  cache_blocks_ = get.cache_blocks_;
+  check_existence_only_ = get.check_existence_only_;
+  consistency_ = get.consistency_;
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
+  return *this;
+}
+
+Get &Get::AddFamily(const std::string &family) {
+  const auto &it = family_map_.find(family);
+
+  /**
+   * Check if any qualifiers are already present or not.
+   * Remove all existing qualifiers if the given family is already present in
+   * the map
+   */
+  if (family_map_.end() != it) {
+    it->second.clear();
+  } else {
+    family_map_[family];
+  }
+  return *this;
+}
+
+Get &Get::AddColumn(const std::string &family, const std::string &qualifier) {
+  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
+
+  /**
+   * Check if any qualifiers are already present or not.
+   * Add only if qualifiers for a given family are not present
+   */
+  if (it == family_map_[family].end()) {
+    family_map_[family].push_back(qualifier);
+  }
+  return *this;
+}
+
+hbase::pb::Consistency Get::Consistency() const { return consistency_; }
+
+Get &Get::SetConsistency(hbase::pb::Consistency consistency) {
+  consistency_ = consistency;
+  return *this;
+}
+
+bool Get::HasFamilies() const { return !family_map_.empty(); }
+
+const std::map<std::string, std::vector<std::string>> &Get::FamilyMap() const {
+  return family_map_;
+}
+
+int Get::MaxVersions() const { return max_versions_; }
+
+Get &Get::SetMaxVersions(int32_t max_versions) {
+  if (0 == max_versions) throw std::runtime_error("max_versions must be positive");
+
+  max_versions_ = max_versions;
+  return *this;
+}
+
+bool Get::CacheBlocks() const { return cache_blocks_; }
+
+Get &Get::SetCacheBlocks(bool cache_blocks) {
+  cache_blocks_ = cache_blocks;
+  return *this;
+}
+
+Get &Get::SetTimeRange(int64_t min_timestamp, int64_t max_timestamp) {
+  tr_.reset(new TimeRange(min_timestamp, max_timestamp));
+  return *this;
+}
+
+Get &Get::SetTimeStamp(int64_t timestamp) {
+  tr_.reset(new TimeRange(timestamp, timestamp + 1));
+  return *this;
+}
+
+const TimeRange &Get::Timerange() const { return *tr_; }
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/client/hbase-configuration-loader.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/client/hbase-configuration-loader.cc b/hbase-native-client/src/hbase/client/hbase-configuration-loader.cc
new file mode 100644
index 0000000..98ef20e
--- /dev/null
+++ b/hbase-native-client/src/hbase/client/hbase-configuration-loader.cc
@@ -0,0 +1,208 @@
+/*
+ * 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 "hbase/client/hbase-configuration-loader.h"
+
+#include <glog/logging.h>
+#include <boost/foreach.hpp>
+#include <boost/property_tree/ptree.hpp>
+#include <boost/property_tree/xml_parser.hpp>
+
+namespace hbase {
+
+bool is_valid_bool(const std::string &raw) {
+  if (raw.empty()) {
+    return false;
+  }
+
+  if (!strcasecmp(raw.c_str(), "true")) {
+    return true;
+  }
+  if (!strcasecmp(raw.c_str(), "false")) {
+    return true;
+  }
+  return false;
+}
+
+bool str_to_bool(const std::string &raw) {
+  if (!strcasecmp(raw.c_str(), "true")) {
+    return true;
+  }
+  return false;
+}
+
+HBaseConfigurationLoader::HBaseConfigurationLoader() {}
+
+HBaseConfigurationLoader::~HBaseConfigurationLoader() {}
+
+void HBaseConfigurationLoader::SetDefaultSearchPath() {
+  /*
+   * Try (in order, taking the first valid one):
+   * $HBASE_CONF_DIR
+   * /etc/hbase/conf
+   *
+   */
+  const char *hadoop_conf_dir_env = getenv("HBASE_CONF");
+  if (hadoop_conf_dir_env) {
+    AddToSearchPath(hadoop_conf_dir_env);
+  } else {
+    AddToSearchPath(kHBaseDefauktConfPath);
+  }
+}
+
+void HBaseConfigurationLoader::ClearSearchPath() { search_paths_.clear(); }
+
+void HBaseConfigurationLoader::SetSearchPath(const std::string &search_path) {
+  search_paths_.clear();
+
+  std::vector<std::string> paths;
+  std::string::size_type start = 0;
+  std::string::size_type end = search_path.find(kSearchPathSeparator);
+
+  while (end != std::string::npos) {
+    paths.push_back(search_path.substr(start, end - start));
+    start = ++end;
+    end = search_path.find(kSearchPathSeparator, start);
+  }
+  paths.push_back(search_path.substr(start, search_path.length()));
+
+  for (auto path : paths) {
+    AddToSearchPath(path);
+  }
+}
+
+void HBaseConfigurationLoader::AddToSearchPath(const std::string &search_path) {
+  if (search_path.empty()) return;
+
+  std::string path_to_add(search_path);
+  if (search_path.back() != kFileSeparator) {
+    path_to_add += kFileSeparator;
+  }
+  if (std::find(search_paths_.begin(), search_paths_.end(), path_to_add) == search_paths_.end())
+    search_paths_.push_back(path_to_add);
+}
+
+void HBaseConfigurationLoader::AddDefaultResources() {
+  resources_.push_back(kHBaseDefaultXml);
+  resources_.push_back(kHBaseSiteXml);
+}
+
+void HBaseConfigurationLoader::AddResources(const std::string &filename) {
+  if (std::find(resources_.begin(), resources_.end(), filename) == resources_.end())
+    resources_.push_back(filename);
+}
+
+optional<Configuration> HBaseConfigurationLoader::LoadDefaultResources() {
+  SetDefaultSearchPath();
+  AddDefaultResources();
+  ConfigMap conf_property;
+  bool success = false;
+  for (auto dir : search_paths_) {
+    for (auto file : resources_) {
+      std::string config_file = dir + file;
+      std::ifstream stream(config_file);
+      if (stream.is_open()) {
+        success |= LoadProperties(config_file, conf_property);
+      } else {
+        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
+      }
+    }
+  }
+  if (success) {
+    return std::experimental::make_optional<Configuration>(Configuration(conf_property));
+  } else {
+    return optional<Configuration>();
+  }
+}
+
+optional<Configuration> HBaseConfigurationLoader::LoadResources(
+    const std::string &search_path, const std::vector<std::string> &resources) {
+  SetSearchPath(search_path);
+  for (const auto &resource : resources) AddResources(resource);
+  ConfigMap conf_property;
+  bool success = false;
+  for (auto dir : search_paths_) {
+    for (auto file : resources_) {
+      std::string config_file = dir + file;
+      std::ifstream stream(config_file);
+      if (stream.is_open()) {
+        success |= LoadProperties(config_file, conf_property);
+      } else {
+        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
+      }
+    }
+  }
+  if (success) {
+    return std::experimental::make_optional<Configuration>(Configuration(conf_property));
+  } else {
+    return optional<Configuration>();
+  }
+}
+
+bool HBaseConfigurationLoader::LoadProperties(const std::string &file, ConfigMap &property_map) {
+  // Create empty property tree object
+  using boost::property_tree::ptree;
+  ptree pt;
+  try {
+    // Load XML file and put contents in a property tree.
+    // If read fails, throw exception.
+    read_xml(file, pt);
+
+    // If configuration key is not found exception is thrown
+    std::string configuration = pt.get<std::string>("configuration");
+
+    // Iterate over configuration section.
+    // Store all found properties in ConfigMap
+    BOOST_FOREACH (ptree::value_type &v, pt.get_child("configuration")) {
+      if ("property" == v.first) {
+        std::string name_node = v.second.get<std::string>("name");
+        std::string value_node = v.second.get<std::string>("value");
+        if ((name_node.size() > 0) && (value_node.size() > 0)) {
+          boost::optional<std::string> final_node = v.second.get_optional<std::string>("final");
+          UpdateMapWithValue(property_map, name_node, value_node, final_node);
+        }
+      }
+    }
+  } catch (std::exception &ex) {
+    DLOG(WARNING) << "Exception in parsing file [" << file << "]:[" << ex.what() << "]";
+    return false;
+  }
+  return true;
+}
+
+bool HBaseConfigurationLoader::UpdateMapWithValue(ConfigMap &map, const std::string &key,
+                                                  const std::string &value,
+                                                  boost::optional<std::string> final_text) {
+  auto map_value = map.find(key);
+  if (map_value != map.end() && map_value->second.final) {
+    return false;
+  }
+
+  bool final_value = false;
+  if (nullptr != final_text.get_ptr()) {
+    if (is_valid_bool(final_text.get())) {
+      final_value = str_to_bool(final_text.get());
+    }
+  }
+
+  map[key].value = value;
+  map[key].final = final_value;
+  return true;
+}
+} /* namespace hbase */


[12/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/raw-async-table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/raw-async-table.h b/hbase-native-client/include/hbase/client/raw-async-table.h
new file mode 100644
index 0000000..9db291e
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/raw-async-table.h
@@ -0,0 +1,114 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Unit.h>
+#include <folly/futures/Future.h>
+#include <chrono>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/client/async-client-scanner.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/async-rpc-retrying-caller-factory.h"
+#include "hbase/client/async-rpc-retrying-caller.h"
+#include "hbase/client/connection-configuration.h"
+#include "hbase/client/delete.h"
+#include "hbase/client/get.h"
+#include "hbase/client/increment.h"
+#include "hbase/client/put.h"
+#include "hbase/client/result.h"
+#include "hbase/client/scan.h"
+
+namespace hbase {
+
+/**
+ * A low level asynchronous table that should not be used by user applications.The implementation
+ * is required to be thread safe.
+ */
+class RawAsyncTable {
+ public:
+  RawAsyncTable(std::shared_ptr<pb::TableName> table_name,
+                std::shared_ptr<AsyncConnection> connection)
+      : connection_(connection),
+        connection_conf_(connection->connection_conf()),
+        table_name_(table_name),
+        rpc_client_(connection->rpc_client()) {
+    default_scanner_caching_ = connection_conf_->scanner_caching();
+    default_scanner_max_result_size_ = connection_conf_->scanner_max_result_size();
+  }
+  virtual ~RawAsyncTable() = default;
+
+  folly::Future<std::shared_ptr<Result>> Get(const hbase::Get& get);
+
+  folly::Future<folly::Unit> Delete(const hbase::Delete& del);
+
+  folly::Future<std::shared_ptr<hbase::Result>> Append(const hbase::Append& append);
+
+  folly::Future<std::shared_ptr<hbase::Result>> Increment(const hbase::Increment& increment);
+
+  folly::Future<folly::Unit> Put(const hbase::Put& put);
+
+  folly::Future<bool> CheckAndPut(const std::string& row, const std::string& family,
+                                  const std::string& qualifier, const std::string& value,
+                                  const hbase::Put& put,
+                                  const pb::CompareType& compare_op = pb::CompareType::EQUAL);
+
+  folly::Future<bool> CheckAndDelete(const std::string& row, const std::string& family,
+                                     const std::string& qualifier, const std::string& value,
+                                     const hbase::Delete& del,
+                                     const pb::CompareType& compare_op = pb::CompareType::EQUAL);
+
+  void Scan(const hbase::Scan& scan, std::shared_ptr<RawScanResultConsumer> consumer);
+
+  void Close() {}
+
+  folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> Get(
+      const std::vector<hbase::Get>& gets);
+  template <typename REQ, typename RESP>
+  folly::Future<std::vector<folly::Try<RESP>>> Batch(const std::vector<REQ>& rows,
+                                                     std::chrono::nanoseconds timeout);
+  folly::Future<std::vector<folly::Try<std::shared_ptr<Result>>>> Put(
+      const std::vector<hbase::Put>& puts);
+
+ private:
+  /* Data */
+  std::shared_ptr<AsyncConnection> connection_;
+  std::shared_ptr<ConnectionConfiguration> connection_conf_;
+  std::shared_ptr<pb::TableName> table_name_;
+  std::shared_ptr<RpcClient> rpc_client_;
+  int32_t default_scanner_caching_;
+  int64_t default_scanner_max_result_size_;
+
+  /* Methods */
+  template <typename REQ, typename PREQ, typename PRESP, typename RESP>
+  folly::Future<RESP> Call(
+      std::shared_ptr<RpcClient> rpc_client, std::shared_ptr<HBaseRpcController> controller,
+      std::shared_ptr<RegionLocation> loc, const REQ& req,
+      const ReqConverter<std::unique_ptr<PREQ>, REQ, std::string> req_converter,
+      const RespConverter<RESP, PRESP> resp_converter);
+
+  template <typename RESP>
+  std::shared_ptr<SingleRequestCallerBuilder<RESP>> CreateCallerBuilder(
+      std::string row, std::chrono::nanoseconds rpc_timeout);
+
+  std::shared_ptr<hbase::Scan> SetDefaultScanConfig(const hbase::Scan& scan);
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/raw-scan-result-consumer.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/raw-scan-result-consumer.h b/hbase-native-client/include/hbase/client/raw-scan-result-consumer.h
new file mode 100644
index 0000000..26ff709
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/raw-scan-result-consumer.h
@@ -0,0 +1,131 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/Logging.h>
+#include <chrono>
+#include <memory>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+enum class ScanControllerState { kInitialized, kSuspended, kTerminated, kDestroyed };
+
+enum class ScanResumerState { kInitialized, kSuspended, kResumed };
+
+/**
+ * Used to resume a scan.
+ */
+class ScanResumer {
+ public:
+  virtual ~ScanResumer() = default;
+
+  /**
+   * Resume the scan. You are free to call it multiple time but only the first call will take
+   * effect.
+   */
+  virtual void Resume() = 0;
+};
+
+/**
+ * Used to suspend or stop a scan.
+ * <p>
+ * Notice that, you should only call the methods below inside onNext or onHeartbeat method. A
+ * IllegalStateException will be thrown if you call them at other places.
+ * <p>
+ * You can only call one of the methods below, i.e., call suspend or terminate(of course you are
+ * free to not call them both), and the methods are not reentrant. A IllegalStateException will be
+ * thrown if you have already called one of the methods.
+ */
+class ScanController {
+ public:
+  virtual ~ScanController() = default;
+
+  /**
+   * Suspend the scan.
+   * <p>
+   * This means we will stop fetching data in background, i.e., will not call onNext any more
+   * before you resume the scan.
+   * @return A resumer used to resume the scan later.
+   */
+  virtual std::shared_ptr<ScanResumer> Suspend() = 0;
+
+  /**
+   * Terminate the scan.
+   * <p>
+   * This is useful when you have got enough results and want to stop the scan in onNext method,
+   * or you want to stop the scan in onHeartbeat method because it has spent too many time.
+   */
+  virtual void Terminate() = 0;
+};
+
+/**
+ * Receives {@link Result} for an asynchronous scan.
+ * <p>
+ * Notice that, the {@link #onNext(Result[], ScanController)} method will be called in the thread
+ * which we send request to HBase service. So if you want the asynchronous scanner fetch data from
+ * HBase in background while you process the returned data, you need to move the processing work to
+ * another thread to make the {@code onNext} call return immediately. And please do NOT do any time
+ * consuming tasks in all methods below unless you know what you are doing.
+ */
+class RawScanResultConsumer {
+ public:
+  virtual ~RawScanResultConsumer() = default;
+
+  /**
+   * Indicate that we have receive some data.
+   * @param results the data fetched from HBase service.
+   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
+   *          instance is only valid within scope of onNext method. You can only call its method in
+   *          onNext, do NOT store it and call it later outside onNext.
+   */
+  virtual void OnNext(const std::vector<std::shared_ptr<Result>> &results,
+                      std::shared_ptr<ScanController> controller) {}
+
+  /**
+   * Indicate that there is an heartbeat message but we have not cumulated enough cells to call
+   * onNext.
+   * <p>
+   * This method give you a chance to terminate a slow scan operation.
+   * @param controller used to suspend or terminate the scan. Notice that the {@code controller}
+   *          instance is only valid within the scope of onHeartbeat method. You can only call its
+   *          method in onHeartbeat, do NOT store it and call it later outside onHeartbeat.
+   */
+  virtual void OnHeartbeat(std::shared_ptr<ScanController> controller) {}
+
+  /**
+   * Indicate that we hit an unrecoverable error and the scan operation is terminated.
+   * <p>
+   * We will not call {@link #onComplete()} after calling {@link #onError(Throwable)}.
+   */
+  virtual void OnError(const folly::exception_wrapper &error) {}
+
+  /**
+   * Indicate that the scan operation is completed normally.
+   */
+  virtual void OnComplete() {}
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/region-location.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/region-location.h b/hbase-native-client/include/hbase/client/region-location.h
new file mode 100644
index 0000000..3eded91
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/region-location.h
@@ -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.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+enum class RegionLocateType { kBefore, kCurrent, kAfter };
+
+/**
+ * @brief class to hold where a region is located.
+ *
+ * This class holds where a region is located, the information about it, the
+ * region name.
+ */
+class RegionLocation {
+ public:
+  /**
+   * Constructor.
+   * @param region_name The region name of this region.
+   * @param ri The decoded RegionInfo of this region.
+   * @param sn The server name of the HBase regionserver thought to be hosting
+   * this region.
+   */
+  RegionLocation(std::string region_name, hbase::pb::RegionInfo ri, hbase::pb::ServerName sn)
+      : region_name_(region_name), ri_(ri), sn_(sn) {}
+
+  /**
+   * Get a reference to the regio info
+   */
+  const hbase::pb::RegionInfo &region_info() const { return ri_; }
+
+  /**
+   * Get a reference to the server name
+   */
+  const hbase::pb::ServerName &server_name() const { return sn_; }
+
+  /**
+   * Get a reference to the region name.
+   */
+  const std::string &region_name() const { return region_name_; }
+
+  /**
+   * Set the servername if the region has moved.
+   */
+  void set_server_name(hbase::pb::ServerName sn) { sn_ = sn; }
+
+  const std::string DebugString() const {
+    return "region_info:" + ri_.ShortDebugString() + ", server_name:" + sn_.ShortDebugString();
+  }
+
+ private:
+  std::string region_name_;
+  hbase::pb::RegionInfo ri_;
+  hbase::pb::ServerName sn_;
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/region-request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/region-request.h b/hbase-native-client/include/hbase/client/region-request.h
new file mode 100644
index 0000000..9e7f85e
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/region-request.h
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <queue>
+#include <vector>
+#include "hbase/client/action.h"
+#include "hbase/client/region-location.h"
+
+namespace hbase {
+
+class RegionRequest {
+ public:
+  // Concurrent
+  using ActionList = std::vector<std::shared_ptr<Action>>;
+  explicit RegionRequest(const std::shared_ptr<hbase::RegionLocation> &region_loc)
+      : region_loc_(region_loc) {}
+  ~RegionRequest() {}
+  void AddAction(std::shared_ptr<Action> action) { actions_.push_back(action); }
+  std::shared_ptr<hbase::RegionLocation> region_location() const { return region_loc_; }
+  const ActionList &actions() const { return actions_; }
+
+ private:
+  std::shared_ptr<hbase::RegionLocation> region_loc_;
+  ActionList actions_;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/region-result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/region-result.h b/hbase-native-client/include/hbase/client/region-result.h
new file mode 100644
index 0000000..87398a2
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/region-result.h
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <map>
+#include <memory>
+#include <string>
+#include <tuple>
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+using ResultOrExceptionTuple =
+    std::tuple<std::shared_ptr<hbase::Result>, std::shared_ptr<folly::exception_wrapper>>;
+
+class RegionResult {
+ public:
+  RegionResult();
+  void AddResultOrException(int32_t index, std::shared_ptr<hbase::Result> result,
+                            std::shared_ptr<folly::exception_wrapper> exc);
+
+  void set_stat(std::shared_ptr<pb::RegionLoadStats> stat);
+
+  int ResultOrExceptionSize() const;
+
+  std::shared_ptr<ResultOrExceptionTuple> ResultOrException(int32_t index) const;
+
+  const std::shared_ptr<pb::RegionLoadStats>& stat() const;
+
+  ~RegionResult();
+
+ private:
+  std::map<int, ResultOrExceptionTuple> result_or_excption_;
+  std::shared_ptr<pb::RegionLoadStats> stat_;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/request-converter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/request-converter.h b/hbase-native-client/include/hbase/client/request-converter.h
new file mode 100644
index 0000000..eef6b5d
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/request-converter.h
@@ -0,0 +1,125 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/connection/request.h"
+#include "hbase/client/action.h"
+#include "hbase/client/append.h"
+#include "hbase/client/cell.h"
+#include "hbase/client/delete.h"
+#include "hbase/client/get.h"
+#include "hbase/client/increment.h"
+#include "hbase/client/mutation.h"
+#include "hbase/client/put.h"
+#include "hbase/client/region-request.h"
+#include "hbase/client/scan.h"
+#include "hbase/client/server-request.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+using MutationType = hbase::pb::MutationProto_MutationType;
+using DeleteType = hbase::pb::MutationProto_DeleteType;
+
+namespace hbase {
+
+using ActionsByRegion = ServerRequest::ActionsByRegion;
+/**
+ * RequestConverter class
+ * This class converts a Client side Get, Scan, Mutate operation to corresponding PB message.
+ */
+class RequestConverter {
+ public:
+  ~RequestConverter();
+
+  /**
+   * @brief Returns a Request object comprising of PB GetRequest created using
+   * passed 'get'
+   * @param get - Get object used for creating GetRequest
+   * @param region_name - table region
+   */
+  static std::unique_ptr<Request> ToGetRequest(const Get &get, const std::string &region_name);
+
+  /**
+   * @brief Returns a Request object comprising of PB ScanRequest created using
+   * passed 'scan'
+   * @param scan - Scan object used for creating ScanRequest
+   * @param region_name - table region
+   */
+  static std::unique_ptr<Request> ToScanRequest(const Scan &scan, const std::string &region_name);
+
+  static std::unique_ptr<Request> ToScanRequest(const Scan &scan, const std::string &region_name,
+                                                int32_t num_rows, bool close_scanner);
+
+  static std::unique_ptr<Request> ToScanRequest(int64_t scanner_id, int32_t num_rows,
+                                                bool close_scanner);
+
+  static std::unique_ptr<Request> ToScanRequest(int64_t scanner_id, int32_t num_rows,
+                                                bool close_scanner, int64_t next_call_seq_id,
+                                                bool renew);
+
+  static std::unique_ptr<Request> ToMultiRequest(const ActionsByRegion &region_requests);
+
+  static std::unique_ptr<Request> DeleteToMutateRequest(const Delete &del,
+                                                        const std::string &region_name);
+
+  static std::unique_ptr<Request> ToMutateRequest(const Put &put, const std::string &region_name);
+
+  static std::unique_ptr<Request> CheckAndPutToMutateRequest(
+      const std::string &row, const std::string &family, const std::string &qualifier,
+      const std::string &value, const pb::CompareType compare_op, const hbase::Put &put,
+      const std::string &region_name);
+
+  static std::unique_ptr<Request> CheckAndDeleteToMutateRequest(
+      const std::string &row, const std::string &family, const std::string &qualifier,
+      const std::string &value, const pb::CompareType compare_op, const hbase::Delete &del,
+      const std::string &region_name);
+
+  static std::unique_ptr<Request> IncrementToMutateRequest(const Increment &incr,
+                                                           const std::string &region_name);
+
+  static std::unique_ptr<pb::MutationProto> ToMutation(const MutationType type,
+                                                       const Mutation &mutation,
+                                                       const int64_t nonce);
+
+  static std::unique_ptr<Request> AppendToMutateRequest(const Append &append,
+                                                        const std::string &region_name);
+
+ private:
+  // Constructor not required. We have all static methods to create PB requests.
+  RequestConverter();
+
+  /**
+   * @brief fills region_specifier with region values.
+   * @param region_name - table region
+   * @param region_specifier - RegionSpecifier to be filled and passed in PB
+   * Request.
+   */
+  static void SetRegion(const std::string &region_name, pb::RegionSpecifier *region_specifier);
+  static std::unique_ptr<hbase::pb::Get> ToGet(const Get &get);
+  static std::unique_ptr<hbase::pb::Scan> ToScan(const Scan &scan);
+  static DeleteType ToDeleteType(const CellType type);
+  static bool IsDelete(const CellType type);
+  static void SetCommonScanRequestFields(std::shared_ptr<hbase::pb::ScanRequest>, bool renew);
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/response-converter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/response-converter.h b/hbase-native-client/include/hbase/client/response-converter.h
new file mode 100644
index 0000000..14757a5
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/response-converter.h
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <vector>
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/client/multi-response.h"
+#include "hbase/client/result.h"
+#include "hbase/client/server-request.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/serde/cell-scanner.h"
+
+namespace hbase {
+
+/**
+ * ResponseConverter class
+ * This class converts a PB Response to corresponding Result or other objects.
+ */
+class ResponseConverter {
+ public:
+  ~ResponseConverter();
+
+  static std::shared_ptr<Result> ToResult(const hbase::pb::Result& result,
+                                          const std::shared_ptr<CellScanner> cell_scanner);
+
+  /**
+   * @brief Returns a Result object created by PB Message in passed Response object.
+   * @param resp - Response object having the PB message.
+   */
+  static std::shared_ptr<hbase::Result> FromGetResponse(const Response& resp);
+
+  static std::shared_ptr<hbase::Result> FromMutateResponse(const Response& resp);
+
+  static bool BoolFromMutateResponse(const Response& resp);
+
+  static std::vector<std::shared_ptr<Result>> FromScanResponse(const Response& resp);
+
+  static std::vector<std::shared_ptr<Result>> FromScanResponse(
+      const std::shared_ptr<pb::ScanResponse> resp, std::shared_ptr<CellScanner> cell_scanner);
+
+  static std::unique_ptr<hbase::MultiResponse> GetResults(
+      std::shared_ptr<Request> req, const Response& resp,
+      const ServerRequest::ActionsByRegion& actions_by_region);
+
+ private:
+  // Constructor not required. We have all static methods to extract response from PB messages.
+  ResponseConverter();
+  static std::shared_ptr<folly::exception_wrapper> GetRemoteException(
+      const hbase::pb::NameBytesPair& exc_resp);
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/result-scanner.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/result-scanner.h b/hbase-native-client/include/hbase/client/result-scanner.h
new file mode 100644
index 0000000..0b22684
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/result-scanner.h
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <functional>
+#include <iterator>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/client/cell.h"
+#include "hbase/client/result.h"
+
+namespace hbase {
+
+/**
+ * Interface for client-side scanning. Use Table to obtain instances.
+ */
+class ResultScanner {
+  // TODO: should we implement forward iterators?
+
+ public:
+  virtual ~ResultScanner() {}
+
+  virtual void Close() = 0;
+
+  virtual std::shared_ptr<Result> Next() = 0;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/result.h b/hbase-native-client/include/hbase/client/result.h
new file mode 100644
index 0000000..e5222b6
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/result.h
@@ -0,0 +1,136 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <functional>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/client/cell.h"
+#include "hbase/utils/optional.h"
+
+namespace hbase {
+
+/**
+ * @brief Map of families to all versions of its qualifiers and values
+ * We need to have a reverse ordered map, when storing TS -> value, so that the
+ * most recent value is stored first
+ */
+using ResultMap =
+    std::map<std::string,
+             std::map<std::string, std::map<int64_t, std::string, std::greater<int64_t> > > >;
+
+class Result {
+ public:
+  /**
+   * Constructors
+   */
+  Result(const std::vector<std::shared_ptr<Cell> > &cells, bool exists, bool stale, bool partial);
+  Result(const Result &result);
+  ~Result();
+
+  /**
+   * @brief Return the vector of Cells backing this Result instance. This vector
+   * will be ordered in the same manner
+   * as the one which was passed while creation of the Result instance.
+   */
+  const std::vector<std::shared_ptr<Cell> > &Cells() const;
+
+  /**
+   * @brief  Return a vector of Cells for the family and qualifier or empty list
+   * if the column
+   * did not exist in the result.
+   * @param family - column family
+   * @param qualifier - column qualifier
+   */
+  std::vector<std::shared_ptr<Cell> > ColumnCells(const std::string &family,
+                                                  const std::string &qualifier) const;
+
+  /**
+   * @brief Returns the Cell for the most recent timestamp for a given family
+   * and qualifier.
+   * Returns map of qualifiers to values, only includes latest values
+   * @param family - column family.
+   * @param qualifier - column qualifier
+   */
+  const std::shared_ptr<Cell> ColumnLatestCell(const std::string &family,
+                                               const std::string &qualifier) const;
+
+  /**
+   * @brief Get the latest version of the specified family and qualifier.
+   * @param family - column family
+   * @param qualifier - column qualifier
+   */
+  optional<std::string> Value(const std::string &family, const std::string &qualifier) const;
+
+  /**
+   * @brief Returns if the underlying Cell vector is empty or not
+   */
+  bool IsEmpty() const;
+
+  /**
+   * @brief Retrieves the row key that corresponds to the row from which this
+   * Result was created.
+   */
+  const std::string &Row() const;
+
+  /**
+   * @brief Returns the size of the underlying Cell vector
+   */
+  int Size() const;
+
+  /**
+   * @brief Map of families to all versions of its qualifiers and values.
+   * Returns a three level Map of the form:
+   * Map<family,Map<qualifier,Map<timestamp,value>>>>
+   * All other map returning methods make use of this map internally
+   * The Map is created when the Result instance is created
+   */
+  ResultMap Map() const;
+
+  /**
+   * @brief Map of qualifiers to values.
+   * Returns a Map of the form: Map<qualifier,value>
+   * @param family - column family to get
+   */
+  std::map<std::string, std::string> FamilyMap(const std::string &family) const;
+
+  std::string DebugString() const;
+
+  bool Exists() const { return exists_; }
+
+  bool Stale() const { return stale_; }
+
+  bool Partial() const { return partial_; }
+
+  /** Returns estimated size of the Result object including deep heap space usage
+   * of its Cells and data. Notice that this is a very rough estimate. */
+  size_t EstimatedSize() const;
+
+ private:
+  bool exists_ = false;
+  bool stale_ = false;
+  bool partial_ = false;
+  std::string row_ = "";
+  std::vector<std::shared_ptr<Cell> > cells_;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/row.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/row.h b/hbase-native-client/include/hbase/client/row.h
new file mode 100644
index 0000000..2c7bdd1
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/row.h
@@ -0,0 +1,62 @@
+/*
+ * 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 <limits>
+#include <stdexcept>
+#include <string>
+
+#pragma once
+
+namespace hbase {
+
+class Row {
+ public:
+  Row() {}
+  explicit Row(const std::string &row) : row_(row) { CheckRow(row_); }
+
+  /**
+   * @brief Returns the row for the Row interface.
+   */
+  const std::string &row() const { return row_; }
+  virtual ~Row() {}
+
+ private:
+  /**
+   * @brief Checks if the row for this Get operation is proper or not
+   * @param row Row to check
+   * @throws std::runtime_error if row is empty or greater than
+   * MAX_ROW_LENGTH(i.e. std::numeric_limits<short>::max())
+   */
+  void CheckRow(const std::string &row) {
+    const int16_t kMaxRowLength = std::numeric_limits<int16_t>::max();
+    size_t row_length = row.size();
+    if (0 == row_length) {
+      throw std::runtime_error("Row length can't be 0");
+    }
+    if (row_length > kMaxRowLength) {
+      throw std::runtime_error("Length of " + row + " is greater than max row size: " +
+                               std::to_string(kMaxRowLength));
+    }
+  }
+
+ protected:
+  std::string row_ = "";
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/scan-result-cache.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/scan-result-cache.h b/hbase-native-client/include/hbase/client/scan-result-cache.h
new file mode 100644
index 0000000..e423108
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/scan-result-cache.h
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Logging.h>
+#include <algorithm>
+#include <chrono>
+#include <iterator>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+class ScanResultCache {
+  // In Java, there are 3 different implementations for this. We are not doing partial results,
+  // or scan batching in native code for now, so this version is simpler and
+  // only deals with giving back complete rows as Result. It is more or less implementation
+  // of CompleteScanResultCache.java
+
+ public:
+  /**
+   * Add the given results to cache and get valid results back.
+   * @param results the results of a scan next. Must not be null.
+   * @param is_hearthbeat indicate whether the results is gotten from a heartbeat response.
+   * @return valid results, never null.
+   */
+  std::vector<std::shared_ptr<Result>> AddAndGet(
+      const std::vector<std::shared_ptr<Result>> &results, bool is_hearthbeat);
+
+  void Clear();
+
+  int64_t num_complete_rows() const { return num_complete_rows_; }
+
+ private:
+  /**
+     * Forms a single result from the partial results in the partialResults list. This method is
+     * useful for reconstructing partial results on the client side.
+     * @param partial_results list of partial results
+     * @return The complete result that is formed by combining all of the partial results together
+     */
+  static std::shared_ptr<Result> CreateCompleteResult(
+      const std::vector<std::shared_ptr<Result>> &partial_results);
+
+  std::shared_ptr<Result> Combine();
+
+  std::vector<std::shared_ptr<Result>> PrependCombined(
+      const std::vector<std::shared_ptr<Result>> &results, int length);
+
+  std::vector<std::shared_ptr<Result>> UpdateNumberOfCompleteResultsAndReturn(
+      const std::shared_ptr<Result> &result);
+
+  std::vector<std::shared_ptr<Result>> UpdateNumberOfCompleteResultsAndReturn(
+      const std::vector<std::shared_ptr<Result>> &results);
+
+ private:
+  std::vector<std::shared_ptr<Result>> partial_results_;
+  int64_t num_complete_rows_ = 0;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/scan.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/scan.h b/hbase-native-client/include/hbase/client/scan.h
new file mode 100644
index 0000000..90daa31
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/scan.h
@@ -0,0 +1,270 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/get.h"
+#include "hbase/client/time-range.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class Scan : public Query {
+ public:
+  /**
+   * @brief Constructors. Create a Scan operation across all rows.
+   */
+  Scan();
+  Scan(const Scan &scan);
+  Scan &operator=(const Scan &scan);
+
+  ~Scan();
+
+  /**
+   * @brief Create a Scan operation starting at the specified row. If the
+   * specified row does not exist,
+   * the Scanner will start from the next closest row after the specified row.
+   * @param start_row - row to start scanner at or after
+   */
+  explicit Scan(const std::string &start_row);
+
+  /**
+   * @brief Create a Scan operation for the range of rows specified.
+   * @param start_row - row to start scanner at or after (inclusive).
+   * @param stop_row - row to stop scanner before (exclusive).
+   */
+  Scan(const std::string &start_row, const std::string &stop_row);
+
+  /**
+   * @brief Builds a scan object with the same specs as get.
+   * @param get - get to model scan after
+   */
+  explicit Scan(const Get &get);
+
+  /**
+   * @brief Get all columns from the specified family.Overrides previous calls
+   * to AddColumn for this family.
+   * @param family - family name
+   */
+  Scan &AddFamily(const std::string &family);
+
+  /**
+   * @brief Get the column from the specified family with the specified
+   * qualifier.Overrides previous calls to AddFamily for this family.
+   * @param family - family name.
+   * @param qualifier - column qualifier.
+   */
+  Scan &AddColumn(const std::string &family, const std::string &qualifier);
+
+  /**
+   * @brief Set whether this scan is a reversed one. This is false by default
+   * which means forward(normal) scan.
+   * @param reversed - if true, scan will be backward order
+   */
+  void SetReversed(bool reversed);
+
+  /**
+   * @brief Get whether this scan is a reversed one. Returns  true if backward
+   * scan, false if forward(default) scan
+   */
+  bool IsReversed() const;
+
+  /**
+   * @brief Set the start row of the scan.If the specified row does not exist,
+   * the Scanner will start from the next closest row after the specified row.
+   * @param start_row - row to start scanner at or after
+   * @throws std::runtime_error if start_row length is 0 or greater than
+   * MAX_ROW_LENGTH
+   */
+  void SetStartRow(const std::string &start_row);
+
+  /**
+   * @brief returns start_row of the Scan.
+   */
+  const std::string &StartRow() const;
+
+  /**
+   * @brief Set the stop row of the scan. The scan will include rows that are
+   * lexicographically less than the provided stop_row.
+   * @param stop_row - row to end at (exclusive)
+   * @throws std::runtime_error if stop_row length is 0 or greater than
+   * MAX_ROW_LENGTH
+   */
+  void SetStopRow(const std::string &stop_row);
+
+  /**
+   * @brief returns stop_row of the Scan.
+   */
+  const std::string &StopRow() const;
+
+  /**
+   * @brief Set the number of rows for caching that will be passed to scanners.
+   * Higher caching values will enable faster scanners but will use more memory.
+   * @param caching - the number of rows for caching.
+   */
+  void SetCaching(int caching);
+
+  /**
+   * @brief caching the number of rows fetched when calling next on a scanner.
+   */
+  int Caching() const;
+
+  /**
+   * @brief Sets the consistency level for this operation.
+   * @param consistency - the consistency level
+   */
+  Scan &SetConsistency(const hbase::pb::Consistency consistency);
+
+  /**
+   * @brief Returns the consistency level for this operation.
+   */
+  hbase::pb::Consistency Consistency() const;
+
+  /**
+   * @brief Set whether blocks should be cached for this Scan.This is true by
+   * default. When true, default settings of the table and family are used (this
+   * will never override caching blocks if the block cache is disabled for that
+   * family or entirely).
+   * @param cache_blocks - if false, default settings are overridden and blocks
+   * will not be cached
+   */
+  void SetCacheBlocks(bool cache_blocks);
+
+  /**
+   * @brief Get whether blocks should be cached for this Scan.
+   */
+  bool CacheBlocks() const;
+
+  /**
+   * @brief Setting whether the caller wants to see the partial results that may
+   * be returned from the server. By default this value is false and the
+   * complete results will be assembled client side before being delivered to
+   * the caller.
+   * @param allow_partial_results - if true partial results will be returned.
+   */
+  void SetAllowPartialResults(bool allow_partial_results);
+
+  /**
+   * @brief true when the constructor of this scan understands that the results
+   * they will see may only represent a partial portion of a row. The entire row
+   * would be retrieved by subsequent calls to ResultScanner.next()
+   */
+  bool AllowPartialResults() const;
+
+  /**
+   * @brief Set the value indicating whether loading CFs on demand should be
+   * allowed (cluster default is false). On-demand CF loading doesn't load
+   * column families until necessary.
+   * @param load_column_families_on_demand
+   */
+  void SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand);
+
+  /**
+   * @brief Get the raw loadColumnFamiliesOnDemand setting.
+   */
+  bool LoadColumnFamiliesOnDemand() const;
+
+  /**
+   * @brief Get up to the specified number of versions of each column if
+   * specified else get default i.e. one.
+   * @param max_versions - maximum versions for each column.
+   */
+  Scan &SetMaxVersions(uint32_t max_versions = 1);
+
+  /**
+   * @brief the max number of versions to fetch
+   */
+  int MaxVersions() const;
+
+  /**
+   * @brief Set the maximum result size. The default is -1; this means that no
+   * specific maximum result size will be set for this scan, and the global
+   * configured value will be used instead. (Defaults to unlimited).
+   * @param The maximum result size in bytes.
+   */
+  void SetMaxResultSize(int64_t max_result_size);
+
+  /**
+   * @brief the maximum result size in bytes.
+   */
+  int64_t MaxResultSize() const;
+
+  /**
+   * @brief Get versions of columns only within the specified timestamp range,
+   * [min_stamp, max_stamp). Note, default maximum versions to return is 1. If
+   * your time range spans more than one version and you want all versions
+   * returned, up the number of versions beyond the default.
+   * @param min_stamp - minimum timestamp value, inclusive.
+   * @param max_stamp - maximum timestamp value, exclusive.
+   */
+  Scan &SetTimeRange(int64_t min_stamp, int64_t max_stamp);
+
+  /**
+   * @brief Get versions of columns with the specified timestamp. Note, default
+   * maximum versions to return is 1. If your time range spans more than one
+   * version and you want all versions returned, up the number of versions
+   * beyond the defaut.
+   * @param timestamp - version timestamp
+   */
+  Scan &SetTimeStamp(int64_t timestamp);
+
+  /**
+   * @brief Return Timerange
+   */
+  const TimeRange &Timerange() const;
+
+  /**
+   * @brief Returns true if family map is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the Scan family map for this Scan operation.
+   */
+  const std::map<std::string, std::vector<std::string>> &FamilyMap() const;
+
+ private:
+  std::string start_row_ = "";
+  std::string stop_row_ = "";
+  uint32_t max_versions_ = 1;
+  int32_t caching_ = -1;
+  int64_t max_result_size_ = -1;
+  bool cache_blocks_ = true;
+  bool load_column_families_on_demand_ = false;
+  bool reversed_ = false;
+  bool allow_partial_results_ = false;
+  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
+  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
+  std::map<std::string, std::vector<std::string>> family_map_;
+
+  /**
+   * @brief Checks for row length validity, throws if length check fails,
+   * returns null otherwise.
+   * @param row - row whose validity needs to be checked
+   * @throws std::runtime_error if row length equals 0 or greater than
+   * std::numeric_limits<short>::max();
+   */
+  void CheckRow(const std::string &row);
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/server-request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/server-request.h b/hbase-native-client/include/hbase/client/server-request.h
new file mode 100644
index 0000000..6ad8c66
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/server-request.h
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <map>
+#include <memory>
+#include <stdexcept>
+#include <string>
+#include "hbase/client/action.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/region-request.h"
+
+namespace hbase {
+
+class ServerRequest {
+ public:
+  // Concurrent
+  using ActionsByRegion = std::map<std::string, std::shared_ptr<RegionRequest>>;
+
+  explicit ServerRequest(std::shared_ptr<RegionLocation> region_location) {
+    auto region_name = region_location->region_name();
+    auto region_request = std::make_shared<RegionRequest>(region_location);
+    actions_by_region_[region_name] = region_request;
+  }
+  ~ServerRequest() {}
+
+  void AddActionsByRegion(std::shared_ptr<RegionLocation> region_location,
+                          std::shared_ptr<Action> action) {
+    auto region_name = region_location->region_name();
+    auto search = actions_by_region_.find(region_name);
+    if (search == actions_by_region_.end()) {
+      auto region_request = std::make_shared<RegionRequest>(region_location);
+      actions_by_region_[region_name] = region_request;
+      actions_by_region_[region_name]->AddAction(action);
+    } else {
+      search->second->AddAction(action);
+    }
+  }
+
+  const ActionsByRegion &actions_by_region() const { return actions_by_region_; }
+
+ private:
+  ActionsByRegion actions_by_region_;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/table.h b/hbase-native-client/include/hbase/client/table.h
new file mode 100644
index 0000000..338d19f
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/table.h
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <chrono>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "hbase/connection/rpc-client.h"
+#include "hbase/client/async-connection.h"
+#include "hbase/client/client.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/get.h"
+#include "hbase/client/location-cache.h"
+#include "hbase/client/put.h"
+#include "hbase/client/raw-async-table.h"
+#include "hbase/client/result-scanner.h"
+#include "hbase/client/result.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+class Client;
+
+class Table {
+ public:
+  /**
+   * Constructors
+   */
+  Table(const pb::TableName &table_name, std::shared_ptr<AsyncConnection> async_connection);
+  ~Table();
+
+  /**
+   * @brief - Returns a Result object for the constructed Get.
+   * @param - get Get object to perform HBase Get operation.
+   */
+  std::shared_ptr<hbase::Result> Get(const hbase::Get &get);
+
+  std::vector<std::shared_ptr<hbase::Result>> Get(const std::vector<hbase::Get> &gets);
+
+  /**
+   * @brief - Puts some data in the table.
+   * @param - put Put object to perform HBase Put operation.
+   */
+  void Put(const hbase::Put &put);
+
+  /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it adds the put.  If the passed value is null, the check
+   * is for the lack of column (ie: non-existance)
+   *
+   * @param row to check
+   * @param family column family to check
+   * @param qualifier column qualifier to check
+   * @param value the expected value
+   * @param put data to put if check succeeds
+   * @param compare_op comparison operator to use
+   * @throws IOException e
+   * @return true if the new put was executed, false otherwise
+   */
+  bool CheckAndPut(const std::string &row, const std::string &family, const std::string &qualifier,
+                   const std::string &value, const hbase::Put &put,
+                   const pb::CompareType &compare_op = pb::CompareType::EQUAL);
+  /**
+   * @brief - Deletes some data in the table.
+   * @param - del Delete object to perform HBase Delete operation.
+   */
+  void Delete(const hbase::Delete &del);
+
+  /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it adds the delete.  If the passed value is null, the
+   * check is for the lack of column (ie: non-existence)
+   *
+   * The expected value argument of this call is on the left and the current
+   * value of the cell is on the right side of the comparison operator.
+   *
+   * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
+   *
+   * @param row to check
+   * @param family column family to check
+   * @param qualifier column qualifier to check
+   * @param compare_op comparison operator to use
+   * @param value the expected value
+   * @param del data to delete if check succeeds
+   * @return true if the new delete was executed, false otherwise
+   */
+  bool CheckAndDelete(const std::string &row, const std::string &family,
+                      const std::string &qualifier, const std::string &value,
+                      const hbase::Delete &del,
+                      const pb::CompareType &compare_op = pb::CompareType::EQUAL);
+
+  /**
+   * @brief - Increments some data in the table.
+   * @param - increment Increment object to perform HBase Increment operation.
+   */
+  std::shared_ptr<hbase::Result> Increment(const hbase::Increment &increment);
+
+  /**
+   * @brief - Appends some data in the table.
+   * @param - append Append object to perform HBase Append operation.
+   */
+  std::shared_ptr<hbase::Result> Append(const hbase::Append &append);
+
+  std::shared_ptr<ResultScanner> Scan(const hbase::Scan &scan);
+
+  /**
+     * @brief - Multi Puts.
+     * @param - puts vector of hbase::Put.
+     */
+  void Put(const std::vector<hbase::Put> &puts);
+  /**
+   * @brief - Close the client connection.
+   */
+  void Close();
+
+  /**
+   * @brief - Get region location for a row in current table.
+   */
+  std::shared_ptr<RegionLocation> GetRegionLocation(const std::string &row);
+
+ private:
+  std::shared_ptr<pb::TableName> table_name_;
+  std::shared_ptr<AsyncConnection> async_connection_;
+  std::shared_ptr<hbase::Configuration> conf_;
+  std::unique_ptr<RawAsyncTable> async_table_;
+
+ private:
+  std::chrono::milliseconds operation_timeout() const;
+
+  int64_t ResultSize2CacheSize(int64_t max_results_size) const;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/time-range.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/time-range.h b/hbase-native-client/include/hbase/client/time-range.h
new file mode 100644
index 0000000..d645ecd
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/time-range.h
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+
+namespace hbase {
+class TimeRange {
+ public:
+  /**
+   * @brief  Default constructor. Represents interval [0,
+   * std::numeric_limits<int64_t>::max())
+   * (allTime)
+   */
+  TimeRange();
+  TimeRange(const TimeRange &tr);
+  TimeRange &operator=(const TimeRange &tr);
+  /**
+   * @brief Represents interval [minStamp, std::numeric_limits<int64_t>::max())
+   * @param minStamp the minimum timestamp value, inclusive
+   */
+  explicit TimeRange(int64_t min_timestamp);
+  /**
+   * @brief Represents interval [minStamp, maxStamp)
+   * @param minStamp the minimum timestamp, inclusive
+   * @param maxStamp the maximum timestamp, exclusive
+   * @throws std::runtime_error if min_timestamp < 0 or max_timestamp < 0 or
+   * max_timestamp < min_timestamp
+   */
+  TimeRange(int64_t min_timestamp, int64_t max_timestamp);
+  int64_t MinTimeStamp() const;
+  int64_t MaxTimeStamp() const;
+  bool IsAllTime() const;
+  ~TimeRange();
+
+ private:
+  int64_t min_timestamp_;
+  int64_t max_timestamp_;
+  bool all_time_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/zk-util.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/zk-util.h b/hbase-native-client/include/hbase/client/zk-util.h
new file mode 100644
index 0000000..5d9a099
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/zk-util.h
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <cstdlib>
+#include <string>
+#include "hbase/client/configuration.h"
+
+namespace hbase {
+
+class ZKUtil {
+ public:
+  static constexpr const char* kHBaseZookeeperQuorum_ = "hbase.zookeeper.quorum";
+  static constexpr const char* kDefHBaseZookeeperQuorum_ = "localhost:2181";
+  static constexpr const char* kHBaseZookeeperClientPort_ = "hbase.zookeeper.property.clientPort";
+  static constexpr const int32_t kDefHBaseZookeeperClientPort_ = 2181;
+  static constexpr const char* kHBaseZnodeParent_ = "zookeeper.znode.parent";
+  static constexpr const char* kDefHBaseZnodeParent_ = "/hbase";
+  static constexpr const char* kHBaseMetaRegionServer_ = "meta-region-server";
+
+  static constexpr const char* kHBaseZookeeperSessionTimeout_ = "zookeeper.session.timeout";
+  static constexpr const int32_t kDefHBaseZookeeperSessionTimeout_ = 90000;
+
+  static std::string ParseZooKeeperQuorum(const hbase::Configuration& conf);
+
+  static std::string MetaZNode(const hbase::Configuration& conf);
+
+  static int32_t SessionTimeout(const hbase::Configuration& conf);
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/BUCK b/hbase-native-client/include/hbase/connection/BUCK
new file mode 100644
index 0000000..9a0b0cf
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/BUCK
@@ -0,0 +1,57 @@
+##
+# 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.
+
+# This is the library dealing with a single connection
+# to a single server.
+cxx_library(
+    name="connection",
+    header_namespace="hbase/connection",
+    exported_headers=[
+        "client-dispatcher.h",
+        "client-handler.h",
+        "sasl-handler.h",
+        "connection-factory.h",
+        "connection-pool.h",
+        "connection-id.h",
+        "pipeline.h",
+        "request.h",
+        "rpc-connection.h",
+        "response.h",
+        "service.h",
+        "rpc-client.h",
+        "sasl-util.h",
+        "rpc-test-server.h",
+        "rpc-test-server-handler.h",
+        "rpc-fault-injector.h",
+        "rpc-fault-injector-inl.h",
+    ],
+    deps=[
+        "//src/hbase/if:if",
+        "//include/hbase/utils:utils",
+        "//include/hbase/serde:serde",
+        "//include/hbase/security:security",
+        "//third-party:folly",
+        "//third-party:wangle",
+        "//include/hbase/exceptions:exceptions",
+    ],
+    compiler_flags=['-Weffc++'],
+    linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
+    exported_linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
+    visibility=[
+        '//include/hbase/client/...',
+        '//src/hbase/connection/...',
+    ],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/client-dispatcher.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/client-dispatcher.h b/hbase-native-client/include/hbase/connection/client-dispatcher.h
new file mode 100644
index 0000000..33384a7
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/client-dispatcher.h
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/Logging.h>
+#include <wangle/service/ClientDispatcher.h>
+
+#include <atomic>
+#include <map>
+#include <memory>
+#include <mutex>
+#include <string>
+
+#include "hbase/connection/pipeline.h"
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/utils/concurrent-map.h"
+
+namespace hbase {
+
+/**
+ * Dispatcher that assigns a call_id and then routes the response back to the
+ * future.
+ */
+class ClientDispatcher
+    : public wangle::ClientDispatcherBase<SerializePipeline, std::unique_ptr<Request>,
+                                          std::unique_ptr<Response>> {
+ public:
+  /** Create a new ClientDispatcher */
+  explicit ClientDispatcher(const std::string &server);
+  /** Read a response off the pipeline. */
+  void read(Context *ctx, std::unique_ptr<Response> in) override;
+  void readException(Context *ctx, folly::exception_wrapper e) override;
+  void readEOF(Context *ctx) override;
+  /** Take a request as a call and send it down the pipeline. */
+  folly::Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> arg) override;
+  /** Close the dispatcher and the associated pipeline. */
+  folly::Future<folly::Unit> close(Context *ctx) override;
+  /** Close the dispatcher and the associated pipeline. */
+  folly::Future<folly::Unit> close() override;
+
+ private:
+  void CloseAndCleanUpCalls();
+
+ private:
+  std::recursive_mutex mutex_;
+  concurrent_map<uint32_t, folly::Promise<std::unique_ptr<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.
+  std::atomic<uint32_t> current_call_id_;
+  std::string server_;
+  bool is_closed_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/client-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/client-handler.h b/hbase-native-client/include/hbase/connection/client-handler.h
new file mode 100644
index 0000000..b3bd2b6
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/client-handler.h
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <wangle/channel/Handler.h>
+
+#include <atomic>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <utility>
+
+#include "hbase/client/configuration.h"
+#include "hbase/exceptions/exception.h"
+#include "hbase/serde/codec.h"
+#include "hbase/serde/rpc-serde.h"
+#include "hbase/utils/concurrent-map.h"
+
+// Forward decs.
+namespace hbase {
+class Request;
+class Response;
+class HeaderInfo;
+}
+namespace google {
+namespace protobuf {
+class Message;
+}
+}
+
+namespace hbase {
+
+/**
+ * wangle::Handler implementation to convert hbase::Request to IOBuf and
+ * convert IOBuf to hbase::Response.
+ *
+ * This class deals with sending the connection header and preamble
+ * on first request.
+ */
+class ClientHandler
+    : public wangle::Handler<std::unique_ptr<folly::IOBuf>, std::unique_ptr<Response>,
+                             std::unique_ptr<Request>, std::unique_ptr<folly::IOBuf>> {
+ public:
+  /**
+   * Create the handler
+   * @param user_name the user name of the user running this process.
+   */
+  ClientHandler(std::string user_name, std::shared_ptr<Codec> codec,
+                std::shared_ptr<Configuration> conf, const std::string &server);
+
+  /**
+   * Get bytes from the wire.
+   * This should be the full message as the length field decoder should be
+   * in the pipeline before this.
+   */
+  void read(Context *ctx, std::unique_ptr<folly::IOBuf> msg) override;
+
+  /**
+   * Write the data down the wire.
+   */
+  folly::Future<folly::Unit> write(Context *ctx, std::unique_ptr<Request> r) override;
+
+ private:
+  std::unique_ptr<std::once_flag> once_flag_;
+  std::string user_name_;
+  RpcSerde serde_;
+  std::string server_;  // for logging
+  std::shared_ptr<Configuration> conf_;
+
+  // in flight requests
+  std::unique_ptr<concurrent_map<uint32_t, std::shared_ptr<google::protobuf::Message>>> resp_msgs_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/connection-factory.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/connection-factory.h b/hbase-native-client/include/hbase/connection/connection-factory.h
new file mode 100644
index 0000000..14b7fda
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/connection-factory.h
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+#include <wangle/service/Service.h>
+
+#include <chrono>
+#include <memory>
+#include <string>
+
+#include "hbase/connection/pipeline.h"
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/connection/service.h"
+#include "hbase/security/user.h"
+
+namespace hbase {
+
+class RpcConnection;
+
+/**
+ * Class to create a ClientBootstrap and turn it into a connected
+ * pipeline.
+ */
+class ConnectionFactory {
+ public:
+  /**
+   * Constructor.
+   * There should only be one ConnectionFactory per client.
+   */
+  ConnectionFactory(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                    std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                    std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
+                    std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
+
+  /** Default Destructor */
+  virtual ~ConnectionFactory() = default;
+
+  /**
+   * Create a BootStrap from which a connection can be made.
+   */
+  virtual std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> MakeBootstrap();
+
+  /**
+   * Connect a ClientBootstrap to a server and return the pipeline.
+   *
+   * This is mostly visible so that mocks can override socket connections.
+   */
+  virtual std::shared_ptr<HBaseService> Connect(
+      std::shared_ptr<RpcConnection> rpc_connection,
+      std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap,
+      const std::string &hostname, uint16_t port);
+
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() { return io_executor_; }
+
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() { return cpu_executor_; }
+
+ private:
+  std::chrono::nanoseconds connect_timeout_;
+  std::shared_ptr<Configuration> conf_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<RpcPipelineFactory> pipeline_factory_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/connection-id.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/connection-id.h b/hbase-native-client/include/hbase/connection/connection-id.h
new file mode 100644
index 0000000..98f5b47
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/connection-id.h
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <boost/functional/hash.hpp>
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "hbase/if/HBase.pb.h"
+#include "hbase/security/user.h"
+
+namespace hbase {
+
+class ConnectionId {
+ public:
+  ConnectionId(const std::string &host, uint16_t port)
+      : ConnectionId(host, port, security::User::defaultUser(), "") {}
+
+  ConnectionId(const std::string &host, uint16_t port, std::shared_ptr<security::User> user)
+      : ConnectionId(host, port, user, "") {}
+
+  ConnectionId(const std::string &host, uint16_t port, std::shared_ptr<security::User> user,
+               const std::string &service_name)
+      : user_(user), service_name_(service_name), host_(host), port_(port) {}
+
+  ConnectionId(const std::string &host, uint16_t port, const std::string &service_name)
+      : user_(security::User::defaultUser()),
+        service_name_(service_name),
+        host_(host),
+        port_(port) {}
+
+  virtual ~ConnectionId() = default;
+
+  std::shared_ptr<security::User> user() const { return user_; }
+  std::string service_name() const { return service_name_; }
+  std::string host() { return host_; }
+  uint16_t port() { return port_; }
+
+ private:
+  std::shared_ptr<security::User> user_;
+  std::string service_name_;
+  std::string host_;
+  uint16_t port_;
+};
+
+/* Equals function for ConnectionId */
+struct ConnectionIdEquals {
+  /** equals */
+  bool operator()(const std::shared_ptr<ConnectionId> &lhs,
+                  const std::shared_ptr<ConnectionId> &rhs) const {
+    return userEquals(lhs->user(), rhs->user()) && lhs->host() == rhs->host() &&
+           lhs->port() == rhs->port() && lhs->service_name() == rhs->service_name();
+  }
+
+ private:
+  bool userEquals(const std::shared_ptr<security::User> &lhs,
+                  const std::shared_ptr<security::User> &rhs) const {
+    return lhs == nullptr ? rhs == nullptr
+                          : (rhs == nullptr ? false : lhs->user_name() == rhs->user_name());
+  }
+};
+
+/** Hash for ConnectionId. */
+struct ConnectionIdHash {
+  /** hash */
+  std::size_t operator()(const std::shared_ptr<ConnectionId> &ci) const {
+    std::size_t h = 0;
+    boost::hash_combine(h, ci->user() == nullptr ? 0 : ci->user()->user_name());
+    boost::hash_combine(h, ci->host());
+    boost::hash_combine(h, ci->port());
+    boost::hash_combine(h, ci->service_name());
+    return h;
+  }
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/connection-pool.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/connection-pool.h b/hbase-native-client/include/hbase/connection/connection-pool.h
new file mode 100644
index 0000000..1198c33
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/connection-pool.h
@@ -0,0 +1,90 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/SharedMutex.h>
+#include <boost/functional/hash.hpp>
+#include <chrono>
+#include <memory>
+#include <mutex>
+#include <unordered_map>
+
+#include "hbase/connection/connection-factory.h"
+#include "hbase/connection/connection-id.h"
+#include "hbase/connection/rpc-connection.h"
+#include "hbase/connection/service.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+/**
+ * @brief Connection pooling for HBase rpc connection.
+ *
+ * This is a thread safe connection pool. It allows getting
+ * a shared rpc connection to HBase servers by connection id.
+ */
+class ConnectionPool {
+ public:
+  /** Create connection pool wit default connection factory */
+  ConnectionPool(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                 std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                 std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
+                 std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
+
+  /**
+   * Constructor that allows specifiying the connetion factory.
+   * This is useful for testing.
+   */
+  explicit ConnectionPool(std::shared_ptr<ConnectionFactory> cf);
+
+  /**
+   * Destructor.
+   * All connections will be close.
+   * All connections will be released
+   */
+  ~ConnectionPool();
+
+  /**
+   * Get a connection to the server name. Start time is ignored.
+   * This can be a blocking operation for a short time.
+   */
+  std::shared_ptr<RpcConnection> GetConnection(std::shared_ptr<ConnectionId> remote_id);
+
+  /**
+   * Close/remove a connection.
+   */
+  void Close(std::shared_ptr<ConnectionId> remote_id);
+
+  /**
+   * Close the Connection Pool
+   */
+  void Close();
+
+ private:
+  std::shared_ptr<RpcConnection> GetCachedConnection(std::shared_ptr<ConnectionId> remote_id);
+  std::shared_ptr<RpcConnection> GetNewConnection(std::shared_ptr<ConnectionId> remote_id);
+  std::unordered_map<std::shared_ptr<ConnectionId>, std::shared_ptr<RpcConnection>,
+                     ConnectionIdHash, ConnectionIdEquals>
+      connections_;
+  folly::SharedMutexWritePriority map_mutex_;
+  std::shared_ptr<ConnectionFactory> cf_;
+  std::shared_ptr<Configuration> conf_;
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/pipeline.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/pipeline.h b/hbase-native-client/include/hbase/connection/pipeline.h
new file mode 100644
index 0000000..63e9492
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/pipeline.h
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/io/IOBufQueue.h>
+#include <wangle/service/Service.h>
+
+#include <memory>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/client/configuration.h"
+#include "hbase/serde/codec.h"
+#include "hbase/utils/user-util.h"
+
+namespace hbase {
+
+/** Pipeline to turn IOBuf into requests */
+using SerializePipeline = wangle::Pipeline<folly::IOBufQueue &, std::unique_ptr<Request>>;
+
+/**
+ * Factory to create new pipelines for HBase RPC's.
+ */
+class RpcPipelineFactory : public wangle::PipelineFactory<SerializePipeline> {
+ public:
+  /**
+   * Constructor. This will create user util.
+   */
+  explicit RpcPipelineFactory(std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf);
+
+  /**
+   * Create a new pipeline.
+   * The pipeline will be:
+   *
+   * - Async Socke Handler
+   * - Event Base Handler
+   * - Length Field Based Frame Decoder
+   * - Client Handler
+   */
+  SerializePipeline::Ptr newPipeline(std::shared_ptr<folly::AsyncTransportWrapper> sock) override;
+
+ private:
+  UserUtil user_util_;
+  std::shared_ptr<Codec> codec_;
+  std::shared_ptr<Configuration> conf_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/connection/request.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/connection/request.h b/hbase-native-client/include/hbase/connection/request.h
new file mode 100644
index 0000000..4b652c0
--- /dev/null
+++ b/hbase-native-client/include/hbase/connection/request.h
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/Conv.h>
+#include <google/protobuf/message.h>
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+namespace hbase {
+
+/**
+ * Main request class.
+ * This holds the request object and the un-filled in approriatley typed
+ * response object.
+ */
+class Request {
+ public:
+  /** Create a request object for a get */
+  static std::unique_ptr<Request> get();
+  /** Create a request object for a mutate */
+  static std::unique_ptr<Request> mutate();
+  /** Create a request object for a scan */
+  static std::unique_ptr<Request> scan();
+  /** Create a request object for a multi */
+  static std::unique_ptr<Request> multi();
+
+  /**
+   * This should be private. Do not use this.
+   *
+   *
+   * Constructor that's public for make_unique. This sets all the messages and
+   * method name.
+   */
+  Request(std::shared_ptr<google::protobuf::Message> req,
+          std::shared_ptr<google::protobuf::Message> resp, std::string method);
+
+  /** Get the call id. */
+  uint32_t call_id() { return call_id_; }
+  /** Set the call id. This should only be set once. */
+  void set_call_id(uint32_t call_id) { call_id_ = call_id; }
+  /** Get the backing request protobuf message. */
+  std::shared_ptr<google::protobuf::Message> req_msg() { return req_msg_; }
+  /** Get the backing response protobuf message. */
+  std::shared_ptr<google::protobuf::Message> resp_msg() { return resp_msg_; }
+  /** Get the method name. This is used to the the receiving rpc server what
+   * method type to decode. */
+  std::string method() { return method_; }
+
+  std::string DebugString() {
+    return "call_id:" + folly::to<std::string>(call_id_) + ", req_msg:" +
+           req_msg_->ShortDebugString() + ", method:" + method_;
+  }
+
+ private:
+  uint32_t call_id_;
+  std::shared_ptr<google::protobuf::Message> req_msg_ = nullptr;
+  std::shared_ptr<google::protobuf::Message> resp_msg_ = nullptr;
+  std::string method_ = "Get";
+};
+}  // namespace hbase


[13/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/configuration.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/configuration.h b/hbase-native-client/include/hbase/client/configuration.h
new file mode 100644
index 0000000..d70941c
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/configuration.h
@@ -0,0 +1,232 @@
+/*
+ * 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.
+ *
+ */
+
+/* Some pieces of code have been added from HDFS-8707 */
+#pragma once
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <experimental/optional>
+
+namespace hbase {
+
+template <class T>
+using optional = std::experimental::optional<T>;
+
+class Configuration {
+ public:
+  /**
+   * @brief Create Configuration object using an empty map. This
+   * object WILL NOT be initialized from reading XML configuration (hbase-site.xml, etc).
+   * Use HBaseConfigurationLoader to initialize the configuration from
+   * hbase-default.xml and hbase-site.xml files.
+   */
+  Configuration();
+
+  ~Configuration();
+
+  /**
+   * @brief Returns value identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. SubstituteVars will be
+   * called for any variable expansion.
+   */
+  std::string Get(const std::string &key, const std::string &default_value) const;
+
+  /**
+   * @brief Returns int32_t identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to int32_t fails
+   */
+  int32_t GetInt(const std::string &key, int32_t default_value) const;
+
+  /**
+   * @brief Returns int64_t identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called and
+   * @throws std::runtime_error if conversion to int64_t fails
+   */
+  int64_t GetLong(const std::string &key, int64_t default_value) const;
+
+  /**
+   * @brief Returns double identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to double fails
+   */
+  double GetDouble(const std::string &key, double default_value) const;
+
+  /**
+   * @brief Returns bool identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to bool fails
+   */
+  bool GetBool(const std::string &key, bool default_value) const;
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void Set(const std::string &key, const std::string &value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetInt(const std::string &key, int32_t value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetLong(const std::string &key, int64_t value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetDouble(const std::string &key, double value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetBool(const std::string &key, bool value);
+
+ private:
+  friend class HBaseConfigurationLoader;
+
+  /* Transparent data holder for property values */
+  /* Same as Jira HDFS-8707 */
+  struct ConfigData {
+    std::string value;
+    bool final;
+    ConfigData() : final(false) {}
+    explicit ConfigData(const std::string &value) : value(value), final(false) {}
+    void operator=(const std::string &new_value) {
+      value = new_value;
+      final = false;
+    }
+  };
+
+  /**
+   * @brief Map which hold configuration properties.
+   */
+  using ConfigMap = std::map<std::string, ConfigData>;
+
+  /**
+   * @brief Create Configuration object using config_map;
+   * @param config_map - Map consisting of properties.
+   */
+  explicit Configuration(ConfigMap &config_map);
+
+  // Property map filled all the properties loaded by ConfigurationLoader
+  ConfigMap hb_property_;
+
+  // Variable expansion depth
+  const int kMaxSubsts = 20;
+
+  /**
+   * @brief returns value for a property identified by key in ConfigMap.
+   * SubstituteVars will be called for any variable expansion.
+   * @param key Key whose value is to be fetched.
+   */
+  optional<std::string> Get(const std::string &key) const;
+
+  /**
+   * @brief returns optional int32_t value identified by the key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to
+   * int32_t using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to int32_t fails
+   */
+  optional<int32_t> GetInt(const std::string &key) const;
+
+  /**
+   * @brief returns optional int64_t value identified by the key in ConfigMap.
+   * Get(key) is called internally to get the string value which is then
+   * converted to int64_t using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to int64_t fails
+   */
+  optional<int64_t> GetLong(const std::string &key) const;
+
+  /**
+   * @brief returns optional double value identified by the key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to
+   * double using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to double fails
+   */
+  optional<double> GetDouble(const std::string &key) const;
+
+  /**
+   * @brief returns optional bool for a property identified by key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to bool
+   * by checking the validity.
+   * @param key Key whose value is to be fetched. Get(key) is called to get the
+   * string value which is then converted to bool.
+   * @throws std::runtime_error if conversion to bool fails
+   */
+  optional<bool> GetBool(const std::string &key) const;
+
+  /**
+   * @brief This method will perform any variable expansion if present.
+   * @param expression expression string to perform substitutions on.
+   * @throws std::runtime_error if MAX_SUBSTS is exhausted and any more
+   * substitutions are reqd to be performed.
+   */
+  std::string SubstituteVars(const std::string &expr) const;
+
+  /**
+   * @brief This method will check if variable expansion has to be performed or
+   * not. Expression will be checked for presence of "${" and "}" to perform
+   * variable expansion.
+   * @param expr Expression on which will be checked for validity.
+   * @param sub_variable Extracted variable from expr which will be checked
+   * against environment value or ConfigMap values.
+   */
+  size_t IsSubVariable(const std::string &expr, std::string &sub_variable) const;
+
+  /**
+   * @brief This method will fetch value for key from environment if present.
+   * @param key key to be fetched from environment.
+   */
+  optional<std::string> GetEnv(const std::string &key) const;
+
+  /**
+   * @brief This method will fetch value for key from ConfigMap if present.
+   * @param key key to be fetched from environment.
+   */
+  optional<std::string> GetProperty(const std::string &key) const;
+};
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/connection-configuration.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/connection-configuration.h b/hbase-native-client/include/hbase/client/connection-configuration.h
new file mode 100644
index 0000000..308e1ba
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/connection-configuration.h
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <chrono>
+#include <climits>
+#include <string>
+
+#include "hbase/client/configuration.h"
+
+namespace hbase {
+
+/**
+ * Timeout configs.
+ */
+class ConnectionConfiguration {
+ public:
+  explicit ConnectionConfiguration(const Configuration& conf) {
+    connect_timeout_ =
+        ToNanos(conf.GetInt(kClientSocketConnectTimeout, kDefaultClientSocketConnectTimeout));
+    meta_operation_timeout_ =
+        ToNanos(conf.GetLong(kClientMetaOperationTimeout, kDefaultClientOperationTimeout));
+    operation_timeout_ =
+        ToNanos(conf.GetLong(kClientOperationTimeout, kDefaultClientOperationTimeout));
+    rpc_timeout_ = ToNanos(conf.GetLong(kRpcTimeout, kDefaultRpcTimeout));
+    read_rpc_timeout_ = ToNanos(conf.GetLong(kRpcReadTimeout, ToMillis(rpc_timeout_)));
+    write_rpc_timeout_ = ToNanos(conf.GetLong(kRpcWriteTimeout, ToMillis(rpc_timeout_)));
+    pause_ = ToNanos(conf.GetLong(kClientPause, kDefaultClientPause));
+    max_retries_ = conf.GetInt(kClientRetriesNumber, kDefaultClientRetriesNumber);
+    start_log_errors_count_ =
+        conf.GetInt(kStartLogErrorsAfterCount, kDefaultStartLogErrorsAfterCount);
+    scan_timeout_ =
+        ToNanos(conf.GetLong(kClientScannerTimeoutPeriod, kDefaultClientScannerTimeoutPeriod));
+    scanner_caching_ = conf.GetInt(kClientScannerCaching, kDefaultClientScannerCaching);
+    scanner_max_result_size_ =
+        conf.GetLong(kClientScannerMaxResultsSize, kDefaultClientScannerMaxResultsSize);
+  }
+
+  // Used by tests
+  ConnectionConfiguration(std::chrono::nanoseconds connect_timeout,
+                          std::chrono::nanoseconds operation_timeout,
+                          std::chrono::nanoseconds rpc_timeout, std::chrono::nanoseconds pause,
+                          uint32_t max_retries, uint32_t start_log_errors_count)
+      : connect_timeout_(connect_timeout),
+        operation_timeout_(operation_timeout),
+        meta_operation_timeout_(operation_timeout),
+        rpc_timeout_(rpc_timeout),
+        read_rpc_timeout_(rpc_timeout),
+        write_rpc_timeout_(rpc_timeout),
+        pause_(pause),
+        max_retries_(max_retries),
+        start_log_errors_count_(start_log_errors_count) {}
+
+  std::chrono::nanoseconds connect_timeout() const { return connect_timeout_; }
+
+  std::chrono::nanoseconds meta_operation_timeout() const { return meta_operation_timeout_; }
+
+  // timeout for a whole operation such as get, put or delete. Notice that scan will not be effected
+  // by this value, see scanTimeoutNs.
+  std::chrono::nanoseconds operation_timeout() const { return operation_timeout_; }
+
+  // timeout for each rpc request. Can be overridden by a more specific config, such as
+  // readRpcTimeout or writeRpcTimeout.
+  std::chrono::nanoseconds rpc_timeout() const { return rpc_timeout_; }
+
+  // timeout for each read rpc request
+  std::chrono::nanoseconds read_rpc_timeout() const { return read_rpc_timeout_; }
+
+  // timeout for each write rpc request
+  std::chrono::nanoseconds write_rpc_timeout() const { return write_rpc_timeout_; }
+
+  std::chrono::nanoseconds pause() const { return pause_; }
+
+  uint32_t max_retries() const { return max_retries_; }
+
+  /** How many retries are allowed before we start to log */
+  uint32_t start_log_errors_count() const { return start_log_errors_count_; }
+
+  // The scan timeout is used as operation timeout for every
+  // operations in a scan, such as openScanner or next.
+  std::chrono::nanoseconds scan_timeout() const { return scan_timeout_; }
+
+  uint32_t scanner_caching() const { return scanner_caching_; }
+
+  uint64_t scanner_max_result_size() const { return scanner_max_result_size_; }
+
+ private:
+  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
+  static constexpr const char* kClientSocketConnectTimeout =
+      "hbase.ipc.client.socket.timeout.connect";
+  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
+  static constexpr const uint32_t kDefaultClientSocketConnectTimeout = 10000;  // 10 secs
+
+  /** Parameter name for HBase client operation timeout. */
+  static constexpr const char* kClientOperationTimeout = "hbase.client.operation.timeout";
+
+  /** Parameter name for HBase client meta operation timeout. */
+  static constexpr const char* kClientMetaOperationTimeout = "hbase.client.meta.operation.timeout";
+
+  /** Default HBase client operation timeout, which is tantamount to a blocking call */
+  static constexpr const uint32_t kDefaultClientOperationTimeout = 1200000;
+
+  /** timeout for each RPC */
+  static constexpr const char* kRpcTimeout = "hbase.rpc.timeout";
+
+  /** timeout for each read RPC */
+  static constexpr const char* kRpcReadTimeout = "hbase.rpc.read.timeout";
+
+  /** timeout for each write RPC */
+  static constexpr const char* kRpcWriteTimeout = "hbase.rpc.write.timeout";
+
+  static constexpr const uint32_t kDefaultRpcTimeout = 60000;
+
+  /**
+    * Parameter name for client pause value, used mostly as value to wait
+    * before running a retry of a failed get, region lookup, etc.
+    */
+  static constexpr const char* kClientPause = "hbase.client.pause";
+
+  static constexpr const uint64_t kDefaultClientPause = 100;
+
+  /**
+   * Parameter name for maximum retries, used as maximum for all retryable
+   * operations such as fetching of the root region from root region server,
+   * getting a cell's value, starting a row update, etc.
+   */
+  static constexpr const char* kClientRetriesNumber = "hbase.client.retries.number";
+
+  static constexpr const uint32_t kDefaultClientRetriesNumber = 35;
+
+  /**
+    * Configure the number of failures after which the client will start logging. A few failures
+    * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable
+    * heuristic for the number of errors we don't log. 9 was chosen because we wait for 1s at
+    * this stage.
+    */
+  static constexpr const char* kStartLogErrorsAfterCount = "hbase.client.start.log.errors.counter";
+  static constexpr const uint32_t kDefaultStartLogErrorsAfterCount = 9;
+
+  /** The client scanner timeout period in milliseconds. */
+  static constexpr const char* kClientScannerTimeoutPeriod = "hbase.client.scanner.timeout.period";
+
+  static constexpr const uint32_t kDefaultClientScannerTimeoutPeriod = 60000;
+
+  /**
+   * Parameter name to set the default scanner caching for all clients.
+   */
+  static constexpr const char* kClientScannerCaching = "hbase.client.scanner.caching";
+
+  static constexpr const uint32_t kDefaultClientScannerCaching = INT_MAX;
+
+  /**
+   * Parameter name for maximum number of bytes returned when calling a scanner's next method.
+   * Controlled by the client.
+   */
+  static constexpr const char* kClientScannerMaxResultsSize =
+      "hbase.client.scanner.max.result.size";
+
+  /**
+   * Maximum number of bytes returned when calling a scanner's next method.
+   * Note that when a single row is larger than this limit the row is still
+   * returned completely.
+   *
+   * The default value is 2MB.
+   */
+  static constexpr const uint64_t kDefaultClientScannerMaxResultsSize = 2 * 1024 * 1024;
+
+  std::chrono::nanoseconds connect_timeout_;
+  std::chrono::nanoseconds meta_operation_timeout_;
+  std::chrono::nanoseconds operation_timeout_;
+  std::chrono::nanoseconds rpc_timeout_;
+  std::chrono::nanoseconds read_rpc_timeout_;
+  std::chrono::nanoseconds write_rpc_timeout_;
+  std::chrono::nanoseconds pause_;
+  uint32_t max_retries_;
+  uint32_t start_log_errors_count_;
+  std::chrono::nanoseconds scan_timeout_;
+  uint32_t scanner_caching_;
+  uint64_t scanner_max_result_size_;
+
+  static std::chrono::nanoseconds ToNanos(const uint64_t& millis) {
+    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::milliseconds(millis));
+  }
+
+  static uint64_t ToMillis(const std::chrono::nanoseconds& nanos) {
+    return std::chrono::duration_cast<std::chrono::milliseconds>(nanos).count();
+  }
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/delete.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/delete.h b/hbase-native-client/include/hbase/client/delete.h
new file mode 100644
index 0000000..5fdbcc1
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/delete.h
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Delete : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  /*
+   * If no further operations are done, this will delete everything
+   * associated with the specified row (all versions of all columns in all
+   * families), with timestamp from current point in time to the past.
+   * Cells defining timestamp for a future point in time
+   * (timestamp > current time) will not be deleted.
+   */
+  explicit Delete(const std::string& row) : Mutation(row) {}
+
+  Delete(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
+  Delete(const Delete& cdelete) : Mutation(cdelete) {}
+  Delete& operator=(const Delete& cdelete) {
+    Mutation::operator=(cdelete);
+    return *this;
+  }
+
+  ~Delete() = default;
+
+  /**
+   *  @brief Add the specified column to this Delete operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   */
+  Delete& AddColumn(const std::string& family, const std::string& qualifier);
+
+  /**
+   *  @brief Add the specified column to this Delete operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   */
+  Delete& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp);
+
+  /**
+   *  @brief Deletes all versions of the specified column
+   *  @param family family name
+   *  @param qualifier column qualifier
+   */
+  Delete& AddColumns(const std::string& family, const std::string& qualifier);
+  /**
+   *  @brief Deletes all versions of the specified column with a timestamp less than
+   * or equal to the specified timestamp
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   */
+  Delete& AddColumns(const std::string& family, const std::string& qualifier, int64_t timestamp);
+  /**
+   *  @brief Add the specified family to this Delete operation.
+   *  @param family family name
+   */
+  Delete& AddFamily(const std::string& family);
+
+  /**
+   *  @brief Deletes all columns of the specified family with a timestamp less than
+   * or equal to the specified timestamp
+   *  @param family family name
+   *  @param timestamp version timestamp
+   */
+  Delete& AddFamily(const std::string& family, int64_t timestamp);
+  /**
+   *  @brief Deletes all columns of the specified family with a timestamp
+   *   equal to the specified timestamp
+   *  @param family family name
+   *  @param timestamp version timestamp
+   */
+  Delete& AddFamilyVersion(const std::string& family, int64_t timestamp);
+  /**
+   * Advanced use only.
+   * Add an existing delete marker to this Delete object.
+   */
+  Delete& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/filter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/filter.h b/hbase-native-client/include/hbase/client/filter.h
new file mode 100644
index 0000000..6ad4443
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/filter.h
@@ -0,0 +1,479 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+#include <set>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "hbase/if/Comparator.pb.h"
+#include "hbase/if/Filter.pb.h"
+#include "hbase/if/HBase.pb.h"
+
+namespace hbase {
+
+/**
+ * In C++ Client, Filter is a thin wrapper for calling filters defined as a Java class. The actual
+ * filtering logic is not implemented here, but this class provides a mechanism to call
+ * pre-existing Filter classes (like KeyOnlyFilter, SingleColumnValueFilter, etc) with your Get or
+ * Scan RPCs. This class can also be used to call custom Filters defined as a Java class, or
+ * pre-existing Filters not defined below. Some of the interfaces depends on protobuf classes
+ * defined in HBase.proto, Filter.proto and Comparator.proto.
+ *
+ * Consult the Java class docs for learning about the various filters and how they work (and filter
+ * arguments).
+ *
+ * Pre-existing Filters can be used like this:
+ *
+ * Get get(row);
+ * get.SetFilter(FilterFactory::ColumnPrefixFilter("foo_"));
+ *
+ * Custom filters can be invoked like this:
+ * Get get(row);
+ * std::string filter_java_class_name = "foo.bar.baz";
+ * auto filter_data = std::make_unique<pb::MyFilter>();
+ * filter_data->set_foo(foo);
+ * get.SetFilter(std::make_unique<Filter>(filter_java_class_name, filter_data));
+ *
+ */
+class Filter {
+ public:
+  Filter(std::string java_class_name, std::unique_ptr<google::protobuf::Message> data)
+      : java_class_name_(java_class_name), data_(std::move(data)) {}
+  virtual ~Filter() {}
+
+  const std::string java_class_name() const { return java_class_name_; }
+
+  const google::protobuf::Message& data() const { return *data_; }
+  /**
+   * Serialize the filter data to the given buffer. Does protobuf encoding by default.
+   * Can be overriden if Filter does not use protobuf.
+   */
+  virtual void Serialize(std::string* buf) const {
+    if (data_ != nullptr) {
+      data_->SerializeToString(buf);
+    }
+  }
+
+  /** Internal method */
+  static std::unique_ptr<pb::Filter> ToProto(const Filter& filter) {
+    auto pb_filter = std::make_unique<pb::Filter>();
+    pb_filter->set_name(filter.java_class_name());
+    filter.Serialize(pb_filter->mutable_serialized_filter());
+    return std::move(pb_filter);
+  }
+
+ private:
+  std::unique_ptr<google::protobuf::Message> data_;
+  std::string java_class_name_;
+};
+
+/**
+ * Comparator for filters. See ByteArrayComparable documentation in Java.
+ */
+class Comparator {
+ public:
+  Comparator(std::string java_class_name, std::unique_ptr<google::protobuf::Message> data)
+      : java_class_name_(java_class_name), data_(std::move(data)) {}
+  virtual ~Comparator() {}
+
+  const std::string java_class_name() const { return java_class_name_; }
+
+  /**
+   * Serialize the Comparator data to the given buffer. Does protobuf encoding by default.
+   * Can be overriden if Comparator does not use protobuf.
+   */
+  virtual void Serialize(std::string* buf) const {
+    if (data_ != nullptr) {
+      data_->SerializeToString(buf);
+    }
+  }
+
+  /** Internal method */
+  static std::unique_ptr<pb::Comparator> ToProto(const Comparator& comparator) {
+    auto pb_comparator = std::make_unique<pb::Comparator>();
+    pb_comparator->set_name(comparator.java_class_name());
+    comparator.Serialize(pb_comparator->mutable_serialized_comparator());
+    return std::move(pb_comparator);
+  }
+
+ private:
+  std::unique_ptr<google::protobuf::Message> data_;
+  std::string java_class_name_;
+};
+
+/**
+ * Used in row range filters
+ */
+struct RowRange {
+  std::string start_row;
+  bool start_row_inclusive;
+  std::string stop_row;
+  bool stop_row_inclusive;
+};
+
+/**
+ * Factory for creating pre-defined filters.
+ */
+class FilterFactory {
+ public:
+  static std::unique_ptr<Filter> ColumnCountGetFilter(uint32_t limit) noexcept {
+    auto data = std::make_unique<pb::ColumnCountGetFilter>();
+    data->set_limit(limit);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnCountGetFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> ColumnPaginationFilter(uint32_t limit, uint32_t offset) noexcept {
+    auto data = std::make_unique<pb::ColumnPaginationFilter>();
+    data->set_limit(limit);
+    data->set_offset(offset);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPaginationFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> ColumnPaginationFilter(uint32_t limit,
+                                                        const std::string& column_offset) noexcept {
+    auto data = std::make_unique<pb::ColumnPaginationFilter>();
+    data->set_limit(limit);
+    data->set_column_offset(column_offset);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPaginationFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> ColumnPrefixFilter(const std::string& prefix) noexcept {
+    auto data = std::make_unique<pb::ColumnPrefixFilter>();
+    data->set_prefix(prefix);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPrefixFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> ColumnRangeFilter(const std::string& min_column,
+                                                   bool min_column_inclusive,
+                                                   const std::string& max_column,
+                                                   bool max_column_inclusive) noexcept {
+    auto data = std::make_unique<pb::ColumnRangeFilter>();
+    data->set_min_column(min_column);
+    data->set_min_column_inclusive(min_column_inclusive);
+    data->set_max_column(max_column);
+    data->set_max_column_inclusive(max_column_inclusive);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnRangeFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<pb::CompareFilter> CompareFilter(pb::CompareType compare_op,
+                                                          const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::CompareFilter>();
+    data->set_compare_op(compare_op);
+    data->set_allocated_comparator(Comparator::ToProto(comparator).release());
+    return std::move(data);
+  }
+
+  /**
+    * Build a dependent column filter with value checking
+    * dependent column varies will be compared using the supplied
+    * compareOp and comparator, for usage of which
+    * refer to {@link CompareFilter}
+    *
+    * @param family dependent column family
+    * @param qualifier dependent column qualifier
+    * @param drop_dependent_column whether the column should be discarded after
+    * @param compare_op comparison op
+    * @param comparator comparator
+    */
+  static std::unique_ptr<Filter> DependentColumnFilter(const std::string& family,
+                                                       const std::string& qualifier,
+                                                       bool drop_dependent_column,
+                                                       pb::CompareType compare_op,
+                                                       const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::DependentColumnFilter>();
+    data->set_column_family(family);
+    data->set_column_qualifier(qualifier);
+    data->set_drop_dependent_column(drop_dependent_column);
+    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.DependentColumnFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FamilyFilter(pb::CompareType compare_op,
+                                              const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::FamilyFilter>();
+    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FamilyFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FilterAllFilter() noexcept {
+    auto data = std::make_unique<pb::FilterAllFilter>();
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FilterAllFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FilterList(
+      pb::FilterList_Operator op, const std::vector<std::unique_ptr<Filter>>& filters) noexcept {
+    auto data = std::make_unique<pb::FilterList>();
+    data->set_operator_(op);
+    for (auto const& f : filters) {
+      data->mutable_filters()->AddAllocated(Filter::ToProto(*f).release());
+    }
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FilterList", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FirstKeyOnlyFilter() noexcept {
+    auto data = std::make_unique<pb::FirstKeyOnlyFilter>();
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FirstKeyValueMatchingQualifiersFilter(
+      const std::set<std::string>& qualifiers) noexcept {
+    auto data = std::make_unique<pb::FirstKeyValueMatchingQualifiersFilter>();
+    for (auto q : qualifiers) {
+      data->add_qualifiers(q);
+    }
+    return std::make_unique<Filter>(
+        "org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> FuzzyRowFilter(
+      const std::vector<std::pair<std::string, std::string>>& fuzzy_keys_data) noexcept {
+    auto data = std::make_unique<pb::FuzzyRowFilter>();
+    for (auto q : fuzzy_keys_data) {
+      auto p = data->add_fuzzy_keys_data();
+      p->set_first(q.first);
+      p->set_second(q.second);
+    }
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FuzzyRowFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> InclusiveStopFilter(const std::string& stop_row_key) noexcept {
+    auto data = std::make_unique<pb::InclusiveStopFilter>();
+    data->set_stop_row_key(stop_row_key);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.InclusiveStopFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> KeyOnlyFilter(bool len_as_val) noexcept {
+    auto data = std::make_unique<pb::KeyOnlyFilter>();
+    data->set_len_as_val(len_as_val);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.KeyOnlyFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> MultipleColumnPrefixFilter(
+      const std::vector<std::string>& sorted_prefixes) noexcept {
+    auto data = std::make_unique<pb::MultipleColumnPrefixFilter>();
+    for (auto p : sorted_prefixes) {
+      data->add_sorted_prefixes(p);
+    }
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> MultiRowRangeFilter(
+      const std::vector<RowRange>& row_ranges) noexcept {
+    auto data = std::make_unique<pb::MultiRowRangeFilter>();
+    for (auto r : row_ranges) {
+      auto range = data->add_row_range_list();
+      range->set_start_row(r.start_row);
+      range->set_start_row_inclusive(r.start_row_inclusive);
+      range->set_stop_row(r.stop_row);
+      range->set_stop_row_inclusive(r.stop_row_inclusive);
+    }
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.MultiRowRangeFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> PageFilter(uint64_t page_size) noexcept {
+    auto data = std::make_unique<pb::PageFilter>();
+    data->set_page_size(page_size);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.PageFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> PrefixFilter(const std::string& prefix) noexcept {
+    auto data = std::make_unique<pb::PrefixFilter>();
+    data->set_prefix(prefix);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.PrefixFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> QualifierFilter(pb::CompareType compare_op,
+                                                 const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::QualifierFilter>();
+    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.QualifierFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> RandomRowFilter(float chance) noexcept {
+    auto data = std::make_unique<pb::RandomRowFilter>();
+    data->set_chance(chance);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.RandomRowFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> RowFilter(pb::CompareType compare_op,
+                                           const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::RowFilter>();
+    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.RowFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> SingleColumnValueExcludeFilter(
+      const std::string& family, const std::string& qualifier, bool filter_if_missing,
+      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::SingleColumnValueExcludeFilter>();
+    auto f = SingleColumnValueFilterProto(family, qualifier, filter_if_missing, latest_version_only,
+                                          compare_op, comparator);
+    data->set_allocated_single_column_value_filter(f.release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<pb::SingleColumnValueFilter> SingleColumnValueFilterProto(
+      const std::string& family, const std::string& qualifier, bool filter_if_missing,
+      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::SingleColumnValueFilter>();
+    data->set_column_family(family);
+    data->set_column_qualifier(qualifier);
+    data->set_compare_op(compare_op);
+    data->set_filter_if_missing(filter_if_missing);
+    data->set_latest_version_only(latest_version_only);
+    data->set_allocated_comparator(Comparator::ToProto(comparator).release());
+    return data;
+  }
+
+  static std::unique_ptr<Filter> SingleColumnValueFilter(
+      const std::string& family, const std::string& qualifier, bool filter_if_missing,
+      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
+    auto data = SingleColumnValueFilterProto(family, qualifier, filter_if_missing,
+                                             latest_version_only, compare_op, comparator);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SingleColumnValueFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> SkipFilter(const Filter& filter) noexcept {
+    auto data = std::make_unique<pb::SkipFilter>();
+    data->set_allocated_filter(Filter::ToProto(filter).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SkipFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> TimestampsFilter(std::vector<uint64_t> timestamps,
+                                                  bool can_hint) noexcept {
+    auto data = std::make_unique<pb::TimestampsFilter>();
+    for (auto t : timestamps) {
+      data->add_timestamps(t);
+    }
+    data->set_can_hint(can_hint);
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.TimestampsFilter",
+                                    std::move(data));
+  }
+
+  static std::unique_ptr<Filter> ValueFilter(pb::CompareType compare_op,
+                                             const Comparator& comparator) noexcept {
+    auto data = std::make_unique<pb::ValueFilter>();
+    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ValueFilter", std::move(data));
+  }
+
+  static std::unique_ptr<Filter> WhileMatchFilter(const Filter& filter) noexcept {
+    auto data = std::make_unique<pb::WhileMatchFilter>();
+    data->set_allocated_filter(Filter::ToProto(filter).release());
+    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.WhileMatchFilter",
+                                    std::move(data));
+  }
+};
+
+/**
+ * Factory for creating pre-defined Comparators.
+ */
+class ComparatorFactory {
+ public:
+  static std::unique_ptr<pb::ByteArrayComparable> ByteArrayComparable(
+      const std::string& value) noexcept {
+    auto data = std::make_unique<pb::ByteArrayComparable>();
+    data->set_value(value);
+    return std::move(data);
+  }
+
+  static std::unique_ptr<Comparator> BinaryComparator(const std::string& value) noexcept {
+    auto data = std::make_unique<pb::BinaryComparator>();
+    data->set_allocated_comparable(ByteArrayComparable(value).release());
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BinaryComparator",
+                                        std::move(data));
+  }
+
+  static std::unique_ptr<Comparator> LongComparator(const std::string& value) noexcept {
+    // TODO: this should take a uint64_t argument, not a byte array.
+    auto data = std::make_unique<pb::LongComparator>();
+    data->set_allocated_comparable(ByteArrayComparable(value).release());
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.LongComparator",
+                                        std::move(data));
+  }
+
+  static std::unique_ptr<Comparator> BinaryPrefixComparator(const std::string& value) noexcept {
+    auto data = std::make_unique<pb::BinaryPrefixComparator>();
+    data->set_allocated_comparable(ByteArrayComparable(value).release());
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BinaryPrefixComparator",
+                                        std::move(data));
+  }
+
+  static std::unique_ptr<Comparator> BitComparator(const std::string& value,
+                                                   pb::BitComparator_BitwiseOp bit_op) noexcept {
+    auto data = std::make_unique<pb::BitComparator>();
+    data->set_allocated_comparable(ByteArrayComparable(value).release());
+    data->set_bitwise_op(bit_op);
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BitComparator",
+                                        std::move(data));
+  }
+
+  static std::unique_ptr<Comparator> NullComparator() noexcept {
+    auto data = std::make_unique<pb::NullComparator>();
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.NullComparator",
+                                        std::move(data));
+  }
+
+  /**
+   * @param pattern a valid regular expression
+   * @param pattern_flags java.util.regex.Pattern flags
+   * @param charset the charset name
+   * @param engine engine implementation type, either JAVA or JONI
+   */
+  static std::unique_ptr<Comparator> RegexStringComparator(
+      const std::string& pattern, int32_t pattern_flags, const std::string& charset = "UTF-8",
+      const std::string& engine = "JAVA") noexcept {
+    auto data = std::make_unique<pb::RegexStringComparator>();
+    data->set_pattern(pattern);
+    data->set_pattern_flags(pattern_flags);
+    data->set_charset(charset);
+    data->set_engine(engine);
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.RegexStringComparator",
+                                        std::move(data));
+  }
+
+  static std::unique_ptr<Comparator> SubstringComparator(const std::string& substr) noexcept {
+    auto data = std::make_unique<pb::SubstringComparator>();
+    data->set_substr(substr);
+    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.SubstringComparator",
+                                        std::move(data));
+  }
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/get.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/get.h b/hbase-native-client/include/hbase/client/get.h
new file mode 100644
index 0000000..b6a11f9
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/get.h
@@ -0,0 +1,132 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/query.h"
+#include "hbase/client/row.h"
+#include "hbase/client/time-range.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class Get : public Row, public Query {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Get(const std::string& row);
+  Get(const Get& cget);
+  Get& operator=(const Get& cget);
+
+  ~Get();
+
+  /**
+   * @brief Returns the maximum number of values to fetch per CF
+   */
+  int MaxVersions() const;
+
+  /**
+   * @brief Get up to the specified number of versions of each column. default
+   * is 1.
+   * @param max_versions max_versons to set
+   */
+  Get& SetMaxVersions(int32_t max_versions = 1);
+
+  /**
+   * @brief Returns whether blocks should be cached for this Get operation.
+   */
+  bool CacheBlocks() const;
+
+  /**
+   * @brief Set whether blocks should be cached for this Get operation.
+   * @param cache_blocks to set
+   */
+  Get& SetCacheBlocks(bool cache_blocks);
+
+  /**
+   * @brief Returns the Get family map for this Get operation. Used
+   * for constructing Scan object with an already constructed Get
+   */
+  const std::map<std::string, std::vector<std::string>>& FamilyMap() const;
+
+  /**
+   * @brief Returns the timerange for this Get
+   */
+  const TimeRange& Timerange() const;
+
+  /**
+   * @brief Get versions of columns only within the specified timestamp range,
+   * [minStamp, maxStamp).
+   * @param minStamp the minimum timestamp, inclusive
+   * @param maxStamp the maximum timestamp, exclusive
+   */
+  Get& SetTimeRange(int64_t min_timestamp, int64_t max_timestamp);
+
+  /**
+   * @brief Get versions of columns with the specified timestamp.
+   * @param The timestamp to be set
+   */
+  Get& SetTimeStamp(int64_t timestamp);
+
+  /**
+   * @brief Get all columns from the specified family.
+   * @param family to be retrieved
+   */
+  Get& AddFamily(const std::string& family);
+
+  /**
+   *  @brief Get the column from the specific family with the specified
+   * qualifier.
+   *  @param family to be retrieved
+   *  @param qualifier to be retrieved
+   */
+  Get& AddColumn(const std::string& family, const std::string& qualifier);
+
+  /**
+   * @brief Returns true if family map is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the consistency level for this Get operation
+   */
+  hbase::pb::Consistency Consistency() const;
+
+  /**
+   * @brief Sets the consistency level for this Get operation
+   * @param Consistency to be set
+   */
+  Get& SetConsistency(hbase::pb::Consistency consistency);
+
+ private:
+  int32_t max_versions_ = 1;
+  bool cache_blocks_ = true;
+  bool check_existence_only_ = false;
+  std::map<std::string, std::vector<std::string>> family_map_;
+  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
+  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/hbase-configuration-loader.h b/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
new file mode 100644
index 0000000..8792aa2
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
@@ -0,0 +1,140 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <boost/optional.hpp>
+
+#include "hbase/client/configuration.h"
+#include "hbase/utils/optional.h"
+
+namespace hbase {
+
+class HBaseConfigurationLoader {
+ public:
+  HBaseConfigurationLoader();
+  ~HBaseConfigurationLoader();
+
+  /**
+   * @brief Creates a Configuration object based on default resources loaded
+   * from default search paths. Default search path will be either $HBASE_CONF
+   * is set or /etc/hbase/conf. Default resources are hbase-default.xml and
+   * hbase-site.xml.SetDefaultSearchPath() and AddDefaultResources() are used
+   * for the same.
+   * Values are loaded in from hbase-default.xml first and then from
+   * hbase-site.xml.
+   * Properties in hbase-site.xml will override the ones in hbase-default.xml
+   * unless marked as final
+   */
+  optional<Configuration> LoadDefaultResources();
+
+  /*
+   * @brief Creates a Configuration object based on resources loaded from search
+   * paths. Search paths are defined in search_path. Values are loaded from
+   * resources and will be overridden unless marked as final
+   * @param search_path - ':' search paths to load resources.
+   * @param resources - list of resources used to load configuration properties.
+   */
+  optional<Configuration> LoadResources(const std::string &search_path,
+                                        const std::vector<std::string> &resources);
+
+ private:
+  using ConfigMap = Configuration::ConfigMap;
+  const std::string kHBaseDefaultXml = "hbase-default.xml";
+  const std::string kHBaseSiteXml = "hbase-site.xml";
+  const std::string kHBaseDefauktConfPath = "/etc/hbase/conf";
+
+  // Adds FILE_SEPARATOR to the search path
+  const char kFileSeparator = '/';
+
+  // Separator using which multiple search paths can be defined.
+  const char kSearchPathSeparator = ':';
+
+  /**
+   * List of paths which will be looked up for loading properties.
+   */
+  std::vector<std::string> search_paths_;
+
+  /**
+   * List of files which will be looked up in search_paths_ to load properties.
+   */
+  std::vector<std::string> resources_;
+
+  /**
+   * @brief This method sets the search path to the default search path (i.e.
+   * "$HBASE_CONF" or "/etc/hbase/conf" if HBASE_CONF is absent)
+   */
+  void SetDefaultSearchPath();
+
+  /**
+   * @brief Clears out the set search path(s)
+   */
+  void ClearSearchPath();
+
+  /**
+   * @brief Sets the search path to ":"-delimited paths, clearing already
+   * defined values
+   * @param search_path Single path or ":"-delimited separated paths
+   */
+  void SetSearchPath(const std::string &search_path);
+
+  /**
+   * @brief Adds an element to the search path if not already present.
+   * @param search_path Path that will be added to load config values
+   */
+  void AddToSearchPath(const std::string &search_path);
+
+  /**
+   * @brief This method will add default resources i.e. hbase-default.xml and
+   * hbase-site.xml to the default search path.
+   */
+  void AddDefaultResources();
+
+  /**
+   * @brief Adds resources to list for loading config values.
+   * @param filename to be added to resources.
+   */
+  void AddResources(const std::string &filename);
+
+  /**
+   * @brief Loads properties in file identified by file in a map identified by
+   * property_map
+   * @param file XML file which defines HBase configuration properties.
+   * @param property_map Property map representing HBase configuration as key
+   * value pairs.
+   * @throws Boost ptree exception if some parsing issue occurs.
+   */
+  bool LoadProperties(const std::string &file, ConfigMap &property_map);
+
+  /**
+   * @brief This method will create a map of the name and properties.
+   * @param map Property map to hold configuration properties.
+   * @param key value of name node.
+   * @param value value of value node.
+   * @param final_text value of final node true or false if present
+   */
+  bool UpdateMapWithValue(ConfigMap &map, const std::string &key, const std::string &value,
+                          boost::optional<std::string> final_text);
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/hbase-rpc-controller.h b/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
new file mode 100644
index 0000000..33f552b
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <google/protobuf/service.h>
+#include <chrono>
+#include <string>
+
+namespace hbase {
+
+class HBaseRpcController : public google::protobuf::RpcController {
+ public:
+  HBaseRpcController() {}
+  virtual ~HBaseRpcController() = default;
+
+  void set_call_timeout(const std::chrono::milliseconds& call_timeout) {
+    // TODO:
+  }
+
+  void Reset() override {}
+
+  bool Failed() const override { return false; }
+
+  folly::exception_wrapper exception() { return exception_; }
+
+  void set_exception(const folly::exception_wrapper& exception) { exception_ = exception; }
+
+  std::string ErrorText() const override { return ""; }
+
+  void StartCancel() override {}
+
+  void SetFailed(const std::string& reason) override {}
+
+  bool IsCanceled() const override { return false; }
+
+  void NotifyOnCancel(google::protobuf::Closure* callback) override {}
+
+ private:
+  folly::exception_wrapper exception_;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/increment.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/increment.h b/hbase-native-client/include/hbase/client/increment.h
new file mode 100644
index 0000000..3169c19
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/increment.h
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Increment : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Increment(const std::string& row) : Mutation(row) {}
+  Increment(const Increment& cincrement) : Mutation(cincrement) {}
+  Increment& operator=(const Increment& cincrement) {
+    Mutation::operator=(cincrement);
+    return *this;
+  }
+
+  ~Increment() = default;
+
+  /**
+   *  @brief Increment the column from the specific family with the specified qualifier
+   * by the specified amount.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param amount amount to increment by
+   */
+  Increment& AddColumn(const std::string& family, const std::string& qualifier, int64_t amount);
+  Increment& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/keyvalue-codec.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/keyvalue-codec.h b/hbase-native-client/include/hbase/client/keyvalue-codec.h
new file mode 100644
index 0000000..5a92c8c
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/keyvalue-codec.h
@@ -0,0 +1,147 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <memory>
+
+#include "hbase/client/cell.h"
+#include "hbase/serde/codec.h"
+
+namespace hbase {
+
+/**
+ * @brief Class for parsing sequence of Cells based on org.apache.hadoop.hbase.KeyValueCodec.java
+ *
+ * KeyValueCodec implements CellScanner interface. Sequence of cells are obtained from cell_block.
+ * We have CreateEncoder and CreateDecoder public methods which will return Encoder/Decoder
+ * instances which will be used to obtain individual cells in cell_block.
+ * Usage:-
+ * 1) Cell Decoding:-
+ * unique_ptr<CellScanner> cell_scanner = KeyValueCodec::CreateDecoder(cell_block, cb_start_offset,
+ *    cb_length);
+ * while (cell_scanner->Advance()) {
+ *  auto current_cell = cell_scanner->Current
+ * }
+ */
+class KeyValueCodec : public Codec {
+ public:
+  /**
+        * Constructor
+        */
+  KeyValueCodec() {}
+
+  std::unique_ptr<Codec::Encoder> CreateEncoder() override { return std::make_unique<KVEncoder>(); }
+  std::unique_ptr<Codec::Decoder> CreateDecoder(std::unique_ptr<folly::IOBuf> cell_block,
+                                                uint32_t offset, uint32_t length) override {
+    return std::make_unique<KVDecoder>(std::move(cell_block), offset, length);
+  }
+
+  /** @brief returns the java class name corresponding to this Codec implementation */
+  virtual const char* java_class_name() const override { return kJavaClassName; }
+
+  static constexpr const char* kJavaClassName = "org.apache.hadoop.hbase.codec.KeyValueCodec";
+
+ private:
+  class KVEncoder : public Codec::Encoder {
+   public:
+    KVEncoder() {}
+
+    void Write(const Cell& cell) {
+      // TODO: Encode Cells using KeyValueCodec wire format
+    }
+
+    void Flush() {}
+  };
+
+  class KVDecoder : public Codec::Decoder {
+   public:
+    KVDecoder(std::unique_ptr<folly::IOBuf> cell_block, uint32_t cell_block_start_offset,
+              uint32_t cell_block_length);
+    ~KVDecoder();
+
+    /**
+     * @brief Overridden from CellScanner. This method parses cell_block and stores the current in
+     * current_cell_. Current cell can be obtained using cell_scanner.Current();
+     */
+    bool Advance();
+
+    /**
+     * @brief returns the current cell
+     */
+    const std::shared_ptr<Cell> Current() const { return current_cell_; }
+
+    /**
+     * @brief returns the total length of cell_meta_block
+     */
+    uint32_t CellBlockLength() const;
+
+   private:
+    std::shared_ptr<Cell> Decode(folly::io::Cursor& cursor);
+
+    /**
+     * Size of boolean in bytes
+     */
+    const int kHBaseSizeOfBoolean_ = sizeof(uint8_t) / sizeof(uint8_t);
+
+    /**
+     * Size of byte in bytes
+     */
+    const uint8_t kHBaseSizeOfByte_ = kHBaseSizeOfBoolean_;
+
+    /**
+     * Size of int in bytes
+     */
+    const uint32_t kHBaseSizeOfInt_ = sizeof(uint32_t) / kHBaseSizeOfByte_;
+
+    /**
+     * Size of long in bytes
+     */
+    const uint64_t kHBaseSizeOfLong_ = sizeof(uint64_t) / kHBaseSizeOfByte_;
+
+    /**
+     * Size of Short in bytes
+     */
+    const uint16_t kHBaseSizeOfShort_ = sizeof(uint16_t) / kHBaseSizeOfByte_;
+
+    const uint32_t kHBaseSizeOfKeyLength_ = kHBaseSizeOfInt_;
+    const uint32_t kHBaseSizeOfValueLength_ = kHBaseSizeOfInt_;
+    const uint16_t kHBaseSizeOfRowLength_ = kHBaseSizeOfShort_;
+    const uint8_t kHBaseSizeOfFamilyLength_ = kHBaseSizeOfByte_;
+    const uint64_t kHBaseSizeOfTimestamp_ = kHBaseSizeOfLong_;
+    const uint8_t kHBaseSizeOfKeyType_ = kHBaseSizeOfByte_;
+    const uint32_t kHBaseSizeOfTimestampAndKey_ = kHBaseSizeOfTimestamp_ + kHBaseSizeOfKeyType_;
+    const uint32_t kHBaseSizeOfKeyInfrastructure_ =
+        kHBaseSizeOfRowLength_ + kHBaseSizeOfFamilyLength_ + kHBaseSizeOfTimestampAndKey_;
+    const uint32_t kHBaseSizeOfKeyValueInfrastructure_ =
+        kHBaseSizeOfKeyLength_ + kHBaseSizeOfValueLength_;
+
+    std::unique_ptr<folly::IOBuf> cell_block_ = nullptr;
+    uint32_t offset_ = 0;
+    uint32_t length_ = 0;
+    uint32_t cur_pos_ = 0;
+    bool end_of_cell_block_ = false;
+
+    std::shared_ptr<Cell> current_cell_ = nullptr;
+  };
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/location-cache.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/location-cache.h b/hbase-native-client/include/hbase/client/location-cache.h
new file mode 100644
index 0000000..a800642
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/location-cache.h
@@ -0,0 +1,219 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <folly/Executor.h>
+#include <folly/SharedMutex.h>
+#include <folly/futures/Future.h>
+#include <folly/futures/SharedPromise.h>
+#include <wangle/concurrent/CPUThreadPoolExecutor.h>
+#include <wangle/concurrent/IOThreadPoolExecutor.h>
+#include <zookeeper/zookeeper.h>
+
+#include <map>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <unordered_map>
+
+#include "hbase/connection/connection-pool.h"
+#include "hbase/client/async-region-locator.h"
+#include "hbase/client/configuration.h"
+#include "hbase/client/meta-utils.h"
+#include "hbase/client/region-location.h"
+#include "hbase/client/zk-util.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+// Forward
+class Request;
+class Response;
+namespace pb {
+class ServerName;
+class TableName;
+}
+
+/** Equals function for TableName (uses namespace and table name) */
+struct TableNameEquals {
+  /** equals */
+  bool operator()(const hbase::pb::TableName &lht, const hbase::pb::TableName &rht) const {
+    return lht.namespace_() == rht.namespace_() && lht.qualifier() == rht.qualifier();
+  }
+};
+
+/** Hash for TableName. */
+struct TableNameHash {
+  /** hash */
+  std::size_t operator()(hbase::pb::TableName const &t) const {
+    std::size_t h = 0;
+    boost::hash_combine(h, t.namespace_());
+    boost::hash_combine(h, t.qualifier());
+    return h;
+  }
+};
+
+// typedefs for location cache
+typedef std::map<std::string, std::shared_ptr<RegionLocation>> PerTableLocationMap;
+typedef std::unordered_map<hbase::pb::TableName, std::shared_ptr<PerTableLocationMap>,
+                           TableNameHash, TableNameEquals>
+    RegionLocationMap;
+
+/**
+ * Class that can look up and cache locations.
+ */
+class LocationCache : public AsyncRegionLocator {
+ public:
+  /**
+   * Constructor.
+   * @param conf Configuration instance to fetch Zookeeper Quorum and Zookeeper Znode.
+   * @param cpu_executor executor used to run non network IO based
+   * continuations.
+   * @param io_executor executor used to talk to the network
+   */
+  LocationCache(std::shared_ptr<hbase::Configuration> conf,
+                std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
+                std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
+                std::shared_ptr<ConnectionPool> cp);
+  /**
+   * Destructor.
+   * This will clean up the zookeeper connections.
+   */
+  ~LocationCache();
+
+  /**
+   * Where is meta hosted.
+   *
+   * TODO: This should be a RegionLocation.
+   */
+  folly::Future<hbase::pb::ServerName> LocateMeta();
+
+  /**
+   * Go read meta and find out where a region is located. Most users should
+   * never call this method directly and should use LocateRegion() instead.
+   *
+   * @param tn Table name of the table to look up. This object must live until
+   * after the future is returned
+   *
+   * @param row of the table to look up. This object must live until after the
+   * future is returned
+   */
+  folly::Future<std::shared_ptr<RegionLocation>> LocateFromMeta(const hbase::pb::TableName &tn,
+                                                                const std::string &row);
+
+  /**
+   * The only method clients should use for meta lookups. If corresponding
+   * location is cached, it's returned from the cache, otherwise lookup
+   * in meta table is done, location is cached and then returned.
+   * It's expected that tiny fraction of invocations incurs meta scan.
+   * This method is to look up non-meta regions; use LocateMeta() to get the
+   * location of hbase:meta region.
+   *
+   * @param tn Table name of the table to look up. This object must live until
+   * after the future is returned
+   *
+   * @param row of the table to look up. This object must live until after the
+   * future is returned
+   */
+  folly::Future<std::shared_ptr<RegionLocation>> LocateRegion(
+      const hbase::pb::TableName &tn, const std::string &row,
+      const RegionLocateType locate_type = RegionLocateType::kCurrent,
+      const int64_t locate_ns = 0) override;
+
+  /**
+   * Remove the cached location of meta.
+   */
+  std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> InvalidateMeta();
+
+  /**
+   * Return cached region location corresponding to this row,
+   * nullptr if this location isn't cached.
+   */
+  std::shared_ptr<RegionLocation> GetCachedLocation(const hbase::pb::TableName &tn,
+                                                    const std::string &row);
+
+  /**
+   * Add non-meta region location in the cache (location of meta itself
+   * is cached separately).
+   */
+  void CacheLocation(const hbase::pb::TableName &tn, const std::shared_ptr<RegionLocation> loc);
+
+  /**
+   * Check if location corresponding to this row key is cached.
+   */
+  bool IsLocationCached(const hbase::pb::TableName &tn, const std::string &row);
+
+  /**
+   * Return cached location for all region of this table.
+   */
+  std::shared_ptr<PerTableLocationMap> GetTableLocations(const hbase::pb::TableName &tn);
+
+  /**
+   * Completely clear location cache.
+   */
+  void ClearCache();
+
+  /**
+   * Clear all cached locations for one table.
+   */
+  void ClearCachedLocations(const hbase::pb::TableName &tn);
+
+  /**
+   * Clear cached region location.
+   */
+  void ClearCachedLocation(const hbase::pb::TableName &tn, const std::string &row);
+
+  /**
+   * Update cached region location, possibly using the information from exception.
+   */
+  void UpdateCachedLocation(const RegionLocation &loc,
+                            const folly::exception_wrapper &error) override;
+
+  const std::string &zk_quorum() { return zk_quorum_; }
+
+ private:
+  void CloseZooKeeperConnection();
+  void EnsureZooKeeperConnection();
+
+ private:
+  void RefreshMetaLocation();
+  hbase::pb::ServerName ReadMetaLocation();
+  std::shared_ptr<RegionLocation> CreateLocation(const Response &resp);
+  std::shared_ptr<hbase::PerTableLocationMap> GetCachedTableLocations(
+      const hbase::pb::TableName &tn);
+  std::shared_ptr<hbase::PerTableLocationMap> GetNewTableLocations(const hbase::pb::TableName &tn);
+
+  /* data */
+  std::shared_ptr<hbase::Configuration> conf_;
+  std::string zk_quorum_;
+  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
+  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
+  std::shared_ptr<folly::SharedPromise<hbase::pb::ServerName>> meta_promise_;
+  std::recursive_mutex meta_lock_;
+  MetaUtil meta_util_;
+  std::shared_ptr<ConnectionPool> cp_;
+
+  // cached region locations
+  RegionLocationMap cached_locations_;
+  folly::SharedMutexWritePriority locations_lock_;
+
+  // TODO: migrate this to a smart pointer with a deleter.
+  zhandle_t *zk_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/meta-utils.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/meta-utils.h b/hbase-native-client/include/hbase/client/meta-utils.h
new file mode 100644
index 0000000..3fc320a
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/meta-utils.h
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/client/region-location.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+/**
+ * @brief Utility for meta operations.
+ */
+class MetaUtil {
+ public:
+  static constexpr const char *kSystemNamespace = "hbase";
+  static constexpr const char *kMetaTableQualifier = "meta";
+  static constexpr const char *kMetaTableName = "hbase:meta";
+  static constexpr const char *kMetaRegion = "1588230740";
+  static constexpr const char *kMetaRegionName = "hbase:meta,,1";
+  static constexpr const char *kCatalogFamily = "info";
+  static constexpr const char *kRegionInfoColumn = "regioninfo";
+  static constexpr const char *kServerColumn = "server";
+
+  MetaUtil();
+
+  /**
+   * Given a table and a row give the row key from which to start a scan to find
+   * region locations.
+   */
+  std::string RegionLookupRowkey(const hbase::pb::TableName &tn, const std::string &row) const;
+
+  /**
+   * Given a row we're trying to access create a request to look up the
+   * location.
+   */
+  std::unique_ptr<Request> MetaRequest(const hbase::pb::TableName tn, const std::string &row) const;
+
+  /**
+   * Return a RegionLocation from the parsed Response
+   */
+  std::shared_ptr<RegionLocation> CreateLocation(const Response &resp,
+                                                 const hbase::pb::TableName &tn);
+
+  /**
+   * Return whether the table is the meta table.
+   */
+  static bool IsMeta(const hbase::pb::TableName &tn);
+
+  const pb::RegionInfo &meta_region_info() const { return meta_region_info_; }
+
+ private:
+  pb::RegionInfo meta_region_info_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/multi-response.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/multi-response.h b/hbase-native-client/include/hbase/client/multi-response.h
new file mode 100644
index 0000000..b9b9a38
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/multi-response.h
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <exception>
+#include <map>
+#include <memory>
+#include <string>
+
+#include "hbase/client/region-result.h"
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class MultiResponse {
+ public:
+  MultiResponse();
+  /**
+   * @brief Returns Number of pairs in this container
+   */
+  int Size() const;
+
+  /**
+   * Add the pair to the container, grouped by the regionName
+   *
+   * @param regionName
+   * @param originalIndex the original index of the Action (request).
+   * @param resOrEx the result or error; will be empty for successful Put and Delete actions.
+   */
+  void AddRegionResult(const std::string& region_name, int32_t original_index,
+                       std::shared_ptr<Result> result,
+                       std::shared_ptr<folly::exception_wrapper> exc);
+
+  void AddRegionException(const std::string& region_name,
+                          std::shared_ptr<folly::exception_wrapper> exception);
+
+  /**
+   * @return the exception for the region, if any. Null otherwise.
+   */
+  std::shared_ptr<folly::exception_wrapper> RegionException(const std::string& region_name) const;
+
+  const std::map<std::string, std::shared_ptr<folly::exception_wrapper>>& RegionExceptions() const;
+
+  void AddStatistic(const std::string& region_name, std::shared_ptr<pb::RegionLoadStats> stat);
+
+  const std::map<std::string, std::shared_ptr<RegionResult>>& RegionResults() const;
+
+  ~MultiResponse();
+
+ private:
+  // map of regionName to map of Results by the original index for that Result
+  std::map<std::string, std::shared_ptr<RegionResult>> results_;
+  /**
+   * The server can send us a failure for the region itself, instead of individual failure.
+   * It's a part of the protobuf definition.
+   */
+  std::map<std::string, std::shared_ptr<folly::exception_wrapper>> exceptions_;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/mutation.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/mutation.h b/hbase-native-client/include/hbase/client/mutation.h
new file mode 100644
index 0000000..f70e2b7
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/mutation.h
@@ -0,0 +1,96 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <limits>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/row.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class Mutation : public Row {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Mutation(const std::string& row);
+  Mutation(const std::string& row, int64_t timestamp);
+  Mutation(const Mutation& cmutation);
+  Mutation& operator=(const Mutation& cmutation);
+
+  virtual ~Mutation() = default;
+
+  /**
+   * @brief Returns the Mutation family map for this operation.
+   */
+  const std::map<std::string, std::vector<std::unique_ptr<Cell>>>& FamilyMap() const {
+    return family_map_;
+  }
+
+  /**
+   * @brief Returns the timerange for this Get
+   */
+  int64_t TimeStamp() const { return timestamp_; }
+
+  /**
+   * @brief Get versions of columns with the specified timestamp.
+   * @param The timestamp to be set
+   */
+  Mutation& SetTimeStamp(int64_t timestamp) {
+    timestamp_ = timestamp;
+    return *this;
+  }
+
+  /**
+   * @brief Returns true if family map is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the durability level for this Mutation operation
+   */
+  pb::MutationProto_Durability Durability() const;
+
+  /**
+   * @brief Sets the durability level for this Mutation operation
+   * @param durability  the durability to be set
+   */
+  Mutation& SetDurability(pb::MutationProto_Durability durability);
+
+ public:
+  static const constexpr int64_t kLatestTimestamp = std::numeric_limits<int64_t>::max();
+
+ protected:
+  std::map<std::string, std::vector<std::unique_ptr<Cell>>> family_map_;
+  pb::MutationProto_Durability durability_ =
+      hbase::pb::MutationProto_Durability::MutationProto_Durability_USE_DEFAULT;
+  int64_t timestamp_ = kLatestTimestamp;
+
+  std::unique_ptr<Cell> CreateCell(const std::string& family, const std::string& qualifier,
+                                   int64_t timestamp, const std::string& value);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/put.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/put.h b/hbase-native-client/include/hbase/client/put.h
new file mode 100644
index 0000000..cb0e17d
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/put.h
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Put : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Put(const std::string& row) : Mutation(row) {}
+  Put(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
+  Put(const Put& cput) : Mutation(cput) {}
+  Put& operator=(const Put& cput) {
+    Mutation::operator=(cput);
+    return *this;
+  }
+
+  ~Put() = default;
+
+  /**
+   *  @brief Add the specified column and value to this Put operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param value column value
+   */
+  Put& AddColumn(const std::string& family, const std::string& qualifier, const std::string& value);
+
+  /**
+   *  @brief Add the specified column and value to this Put operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   *  @param value column value
+   */
+  Put& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp,
+                 const std::string& value);
+
+  Put& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/query.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/query.h b/hbase-native-client/include/hbase/client/query.h
new file mode 100644
index 0000000..76f613e
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/query.h
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+
+#include "hbase/client/filter.h"
+
+namespace hbase {
+
+/**
+ * Base class for read RPC calls (Get / Scan).
+ */
+class Query {
+ public:
+  Query() = default;
+  Query(const Query &query) {
+    // filter can be a custom subclass of Filter, so we do not do a deep copy here.
+    filter_ = query.filter_;
+  }
+
+  Query &operator=(const Query &query) {
+    filter_ = query.filter_;
+    return *this;
+  }
+
+  virtual ~Query() {}
+
+  void SetFilter(std::shared_ptr<Filter> filter) { filter_ = filter; }
+
+  const std::shared_ptr<Filter> filter() const { return filter_; }
+
+ protected:
+  std::shared_ptr<Filter> filter_ = nullptr;
+};
+
+}  // namespace hbase


[20/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/filter.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/filter.h b/hbase-native-client/core/filter.h
deleted file mode 100644
index 1e870f9..0000000
--- a/hbase-native-client/core/filter.h
+++ /dev/null
@@ -1,479 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <set>
-#include <string>
-#include <utility>
-#include <vector>
-
-#include "if/Comparator.pb.h"
-#include "if/Filter.pb.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-/**
- * In C++ Client, Filter is a thin wrapper for calling filters defined as a Java class. The actual
- * filtering logic is not implemented here, but this class provides a mechanism to call
- * pre-existing Filter classes (like KeyOnlyFilter, SingleColumnValueFilter, etc) with your Get or
- * Scan RPCs. This class can also be used to call custom Filters defined as a Java class, or
- * pre-existing Filters not defined below. Some of the interfaces depends on protobuf classes
- * defined in HBase.proto, Filter.proto and Comparator.proto.
- *
- * Consult the Java class docs for learning about the various filters and how they work (and filter
- * arguments).
- *
- * Pre-existing Filters can be used like this:
- *
- * Get get(row);
- * get.SetFilter(FilterFactory::ColumnPrefixFilter("foo_"));
- *
- * Custom filters can be invoked like this:
- * Get get(row);
- * std::string filter_java_class_name = "foo.bar.baz";
- * auto filter_data = std::make_unique<pb::MyFilter>();
- * filter_data->set_foo(foo);
- * get.SetFilter(std::make_unique<Filter>(filter_java_class_name, filter_data));
- *
- */
-class Filter {
- public:
-  Filter(std::string java_class_name, std::unique_ptr<google::protobuf::Message> data)
-      : java_class_name_(java_class_name), data_(std::move(data)) {}
-  virtual ~Filter() {}
-
-  const std::string java_class_name() const { return java_class_name_; }
-
-  const google::protobuf::Message& data() const { return *data_; }
-  /**
-   * Serialize the filter data to the given buffer. Does protobuf encoding by default.
-   * Can be overriden if Filter does not use protobuf.
-   */
-  virtual void Serialize(std::string* buf) const {
-    if (data_ != nullptr) {
-      data_->SerializeToString(buf);
-    }
-  }
-
-  /** Internal method */
-  static std::unique_ptr<pb::Filter> ToProto(const Filter& filter) {
-    auto pb_filter = std::make_unique<pb::Filter>();
-    pb_filter->set_name(filter.java_class_name());
-    filter.Serialize(pb_filter->mutable_serialized_filter());
-    return std::move(pb_filter);
-  }
-
- private:
-  std::unique_ptr<google::protobuf::Message> data_;
-  std::string java_class_name_;
-};
-
-/**
- * Comparator for filters. See ByteArrayComparable documentation in Java.
- */
-class Comparator {
- public:
-  Comparator(std::string java_class_name, std::unique_ptr<google::protobuf::Message> data)
-      : java_class_name_(java_class_name), data_(std::move(data)) {}
-  virtual ~Comparator() {}
-
-  const std::string java_class_name() const { return java_class_name_; }
-
-  /**
-   * Serialize the Comparator data to the given buffer. Does protobuf encoding by default.
-   * Can be overriden if Comparator does not use protobuf.
-   */
-  virtual void Serialize(std::string* buf) const {
-    if (data_ != nullptr) {
-      data_->SerializeToString(buf);
-    }
-  }
-
-  /** Internal method */
-  static std::unique_ptr<pb::Comparator> ToProto(const Comparator& comparator) {
-    auto pb_comparator = std::make_unique<pb::Comparator>();
-    pb_comparator->set_name(comparator.java_class_name());
-    comparator.Serialize(pb_comparator->mutable_serialized_comparator());
-    return std::move(pb_comparator);
-  }
-
- private:
-  std::unique_ptr<google::protobuf::Message> data_;
-  std::string java_class_name_;
-};
-
-/**
- * Used in row range filters
- */
-struct RowRange {
-  std::string start_row;
-  bool start_row_inclusive;
-  std::string stop_row;
-  bool stop_row_inclusive;
-};
-
-/**
- * Factory for creating pre-defined filters.
- */
-class FilterFactory {
- public:
-  static std::unique_ptr<Filter> ColumnCountGetFilter(uint32_t limit) noexcept {
-    auto data = std::make_unique<pb::ColumnCountGetFilter>();
-    data->set_limit(limit);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnCountGetFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> ColumnPaginationFilter(uint32_t limit, uint32_t offset) noexcept {
-    auto data = std::make_unique<pb::ColumnPaginationFilter>();
-    data->set_limit(limit);
-    data->set_offset(offset);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPaginationFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> ColumnPaginationFilter(uint32_t limit,
-                                                        const std::string& column_offset) noexcept {
-    auto data = std::make_unique<pb::ColumnPaginationFilter>();
-    data->set_limit(limit);
-    data->set_column_offset(column_offset);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPaginationFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> ColumnPrefixFilter(const std::string& prefix) noexcept {
-    auto data = std::make_unique<pb::ColumnPrefixFilter>();
-    data->set_prefix(prefix);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnPrefixFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> ColumnRangeFilter(const std::string& min_column,
-                                                   bool min_column_inclusive,
-                                                   const std::string& max_column,
-                                                   bool max_column_inclusive) noexcept {
-    auto data = std::make_unique<pb::ColumnRangeFilter>();
-    data->set_min_column(min_column);
-    data->set_min_column_inclusive(min_column_inclusive);
-    data->set_max_column(max_column);
-    data->set_max_column_inclusive(max_column_inclusive);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ColumnRangeFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<pb::CompareFilter> CompareFilter(pb::CompareType compare_op,
-                                                          const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::CompareFilter>();
-    data->set_compare_op(compare_op);
-    data->set_allocated_comparator(Comparator::ToProto(comparator).release());
-    return std::move(data);
-  }
-
-  /**
-    * Build a dependent column filter with value checking
-    * dependent column varies will be compared using the supplied
-    * compareOp and comparator, for usage of which
-    * refer to {@link CompareFilter}
-    *
-    * @param family dependent column family
-    * @param qualifier dependent column qualifier
-    * @param drop_dependent_column whether the column should be discarded after
-    * @param compare_op comparison op
-    * @param comparator comparator
-    */
-  static std::unique_ptr<Filter> DependentColumnFilter(const std::string& family,
-                                                       const std::string& qualifier,
-                                                       bool drop_dependent_column,
-                                                       pb::CompareType compare_op,
-                                                       const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::DependentColumnFilter>();
-    data->set_column_family(family);
-    data->set_column_qualifier(qualifier);
-    data->set_drop_dependent_column(drop_dependent_column);
-    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.DependentColumnFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FamilyFilter(pb::CompareType compare_op,
-                                              const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::FamilyFilter>();
-    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FamilyFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FilterAllFilter() noexcept {
-    auto data = std::make_unique<pb::FilterAllFilter>();
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FilterAllFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FilterList(
-      pb::FilterList_Operator op, const std::vector<std::unique_ptr<Filter>>& filters) noexcept {
-    auto data = std::make_unique<pb::FilterList>();
-    data->set_operator_(op);
-    for (auto const& f : filters) {
-      data->mutable_filters()->AddAllocated(Filter::ToProto(*f).release());
-    }
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FilterList", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FirstKeyOnlyFilter() noexcept {
-    auto data = std::make_unique<pb::FirstKeyOnlyFilter>();
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FirstKeyValueMatchingQualifiersFilter(
-      const std::set<std::string>& qualifiers) noexcept {
-    auto data = std::make_unique<pb::FirstKeyValueMatchingQualifiersFilter>();
-    for (auto q : qualifiers) {
-      data->add_qualifiers(q);
-    }
-    return std::make_unique<Filter>(
-        "org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> FuzzyRowFilter(
-      const std::vector<std::pair<std::string, std::string>>& fuzzy_keys_data) noexcept {
-    auto data = std::make_unique<pb::FuzzyRowFilter>();
-    for (auto q : fuzzy_keys_data) {
-      auto p = data->add_fuzzy_keys_data();
-      p->set_first(q.first);
-      p->set_second(q.second);
-    }
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.FuzzyRowFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> InclusiveStopFilter(const std::string& stop_row_key) noexcept {
-    auto data = std::make_unique<pb::InclusiveStopFilter>();
-    data->set_stop_row_key(stop_row_key);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.InclusiveStopFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> KeyOnlyFilter(bool len_as_val) noexcept {
-    auto data = std::make_unique<pb::KeyOnlyFilter>();
-    data->set_len_as_val(len_as_val);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.KeyOnlyFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> MultipleColumnPrefixFilter(
-      const std::vector<std::string>& sorted_prefixes) noexcept {
-    auto data = std::make_unique<pb::MultipleColumnPrefixFilter>();
-    for (auto p : sorted_prefixes) {
-      data->add_sorted_prefixes(p);
-    }
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> MultiRowRangeFilter(
-      const std::vector<RowRange>& row_ranges) noexcept {
-    auto data = std::make_unique<pb::MultiRowRangeFilter>();
-    for (auto r : row_ranges) {
-      auto range = data->add_row_range_list();
-      range->set_start_row(r.start_row);
-      range->set_start_row_inclusive(r.start_row_inclusive);
-      range->set_stop_row(r.stop_row);
-      range->set_stop_row_inclusive(r.stop_row_inclusive);
-    }
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.MultiRowRangeFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> PageFilter(uint64_t page_size) noexcept {
-    auto data = std::make_unique<pb::PageFilter>();
-    data->set_page_size(page_size);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.PageFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> PrefixFilter(const std::string& prefix) noexcept {
-    auto data = std::make_unique<pb::PrefixFilter>();
-    data->set_prefix(prefix);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.PrefixFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> QualifierFilter(pb::CompareType compare_op,
-                                                 const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::QualifierFilter>();
-    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.QualifierFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> RandomRowFilter(float chance) noexcept {
-    auto data = std::make_unique<pb::RandomRowFilter>();
-    data->set_chance(chance);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.RandomRowFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> RowFilter(pb::CompareType compare_op,
-                                           const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::RowFilter>();
-    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.RowFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> SingleColumnValueExcludeFilter(
-      const std::string& family, const std::string& qualifier, bool filter_if_missing,
-      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::SingleColumnValueExcludeFilter>();
-    auto f = SingleColumnValueFilterProto(family, qualifier, filter_if_missing, latest_version_only,
-                                          compare_op, comparator);
-    data->set_allocated_single_column_value_filter(f.release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<pb::SingleColumnValueFilter> SingleColumnValueFilterProto(
-      const std::string& family, const std::string& qualifier, bool filter_if_missing,
-      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::SingleColumnValueFilter>();
-    data->set_column_family(family);
-    data->set_column_qualifier(qualifier);
-    data->set_compare_op(compare_op);
-    data->set_filter_if_missing(filter_if_missing);
-    data->set_latest_version_only(latest_version_only);
-    data->set_allocated_comparator(Comparator::ToProto(comparator).release());
-    return data;
-  }
-
-  static std::unique_ptr<Filter> SingleColumnValueFilter(
-      const std::string& family, const std::string& qualifier, bool filter_if_missing,
-      bool latest_version_only, pb::CompareType compare_op, const Comparator& comparator) noexcept {
-    auto data = SingleColumnValueFilterProto(family, qualifier, filter_if_missing,
-                                             latest_version_only, compare_op, comparator);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SingleColumnValueFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> SkipFilter(const Filter& filter) noexcept {
-    auto data = std::make_unique<pb::SkipFilter>();
-    data->set_allocated_filter(Filter::ToProto(filter).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.SkipFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> TimestampsFilter(std::vector<uint64_t> timestamps,
-                                                  bool can_hint) noexcept {
-    auto data = std::make_unique<pb::TimestampsFilter>();
-    for (auto t : timestamps) {
-      data->add_timestamps(t);
-    }
-    data->set_can_hint(can_hint);
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.TimestampsFilter",
-                                    std::move(data));
-  }
-
-  static std::unique_ptr<Filter> ValueFilter(pb::CompareType compare_op,
-                                             const Comparator& comparator) noexcept {
-    auto data = std::make_unique<pb::ValueFilter>();
-    data->set_allocated_compare_filter(CompareFilter(compare_op, comparator).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.ValueFilter", std::move(data));
-  }
-
-  static std::unique_ptr<Filter> WhileMatchFilter(const Filter& filter) noexcept {
-    auto data = std::make_unique<pb::WhileMatchFilter>();
-    data->set_allocated_filter(Filter::ToProto(filter).release());
-    return std::make_unique<Filter>("org.apache.hadoop.hbase.filter.WhileMatchFilter",
-                                    std::move(data));
-  }
-};
-
-/**
- * Factory for creating pre-defined Comparators.
- */
-class ComparatorFactory {
- public:
-  static std::unique_ptr<pb::ByteArrayComparable> ByteArrayComparable(
-      const std::string& value) noexcept {
-    auto data = std::make_unique<pb::ByteArrayComparable>();
-    data->set_value(value);
-    return std::move(data);
-  }
-
-  static std::unique_ptr<Comparator> BinaryComparator(const std::string& value) noexcept {
-    auto data = std::make_unique<pb::BinaryComparator>();
-    data->set_allocated_comparable(ByteArrayComparable(value).release());
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BinaryComparator",
-                                        std::move(data));
-  }
-
-  static std::unique_ptr<Comparator> LongComparator(const std::string& value) noexcept {
-    // TODO: this should take a uint64_t argument, not a byte array.
-    auto data = std::make_unique<pb::LongComparator>();
-    data->set_allocated_comparable(ByteArrayComparable(value).release());
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.LongComparator",
-                                        std::move(data));
-  }
-
-  static std::unique_ptr<Comparator> BinaryPrefixComparator(const std::string& value) noexcept {
-    auto data = std::make_unique<pb::BinaryPrefixComparator>();
-    data->set_allocated_comparable(ByteArrayComparable(value).release());
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BinaryPrefixComparator",
-                                        std::move(data));
-  }
-
-  static std::unique_ptr<Comparator> BitComparator(const std::string& value,
-                                                   pb::BitComparator_BitwiseOp bit_op) noexcept {
-    auto data = std::make_unique<pb::BitComparator>();
-    data->set_allocated_comparable(ByteArrayComparable(value).release());
-    data->set_bitwise_op(bit_op);
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.BitComparator",
-                                        std::move(data));
-  }
-
-  static std::unique_ptr<Comparator> NullComparator() noexcept {
-    auto data = std::make_unique<pb::NullComparator>();
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.NullComparator",
-                                        std::move(data));
-  }
-
-  /**
-   * @param pattern a valid regular expression
-   * @param pattern_flags java.util.regex.Pattern flags
-   * @param charset the charset name
-   * @param engine engine implementation type, either JAVA or JONI
-   */
-  static std::unique_ptr<Comparator> RegexStringComparator(
-      const std::string& pattern, int32_t pattern_flags, const std::string& charset = "UTF-8",
-      const std::string& engine = "JAVA") noexcept {
-    auto data = std::make_unique<pb::RegexStringComparator>();
-    data->set_pattern(pattern);
-    data->set_pattern_flags(pattern_flags);
-    data->set_charset(charset);
-    data->set_engine(engine);
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.RegexStringComparator",
-                                        std::move(data));
-  }
-
-  static std::unique_ptr<Comparator> SubstringComparator(const std::string& substr) noexcept {
-    auto data = std::make_unique<pb::SubstringComparator>();
-    data->set_substr(substr);
-    return std::make_unique<Comparator>("org.apache.hadoop.hbase.filter.SubstringComparator",
-                                        std::move(data));
-  }
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/get-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get-test.cc b/hbase-native-client/core/get-test.cc
deleted file mode 100644
index 4a44a26..0000000
--- a/hbase-native-client/core/get-test.cc
+++ /dev/null
@@ -1,221 +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/get.h"
-#include "core/cell.h"
-
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-
-using hbase::Cell;
-using hbase::Get;
-
-const int NUMBER_OF_GETS = 5;
-
-void CheckFamilies(Get &get) {
-  EXPECT_EQ(false, get.HasFamilies());
-  get.AddFamily("family-1");
-  EXPECT_EQ(true, get.HasFamilies());
-  EXPECT_EQ(1, get.FamilyMap().size());
-  for (const auto &family : get.FamilyMap()) {
-    EXPECT_STREQ("family-1", family.first.c_str());
-    EXPECT_EQ(0, family.second.size());
-  }
-  // Not allowed to add the same CF.
-  get.AddFamily("family-1");
-  EXPECT_EQ(1, get.FamilyMap().size());
-  get.AddFamily("family-2");
-  EXPECT_EQ(2, get.FamilyMap().size());
-  get.AddFamily("family-3");
-  EXPECT_EQ(3, get.FamilyMap().size());
-  int i = 1;
-  for (const auto &family : get.FamilyMap()) {
-    std::string family_name = "family-" + std::to_string(i);
-    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
-    EXPECT_EQ(0, family.second.size());
-    i += 1;
-  }
-
-  get.AddColumn("family-1", "column-1");
-  get.AddColumn("family-1", "column-2");
-  get.AddColumn("family-1", "");
-  get.AddColumn("family-1", "column-3");
-  get.AddColumn("family-2", "column-X");
-
-  EXPECT_EQ(3, get.FamilyMap().size());
-  auto it = get.FamilyMap().begin();
-  EXPECT_STREQ("family-1", it->first.c_str());
-  EXPECT_EQ(4, it->second.size());
-  EXPECT_STREQ("column-1", it->second[0].c_str());
-  EXPECT_STREQ("column-2", it->second[1].c_str());
-  EXPECT_STREQ("", it->second[2].c_str());
-  EXPECT_STREQ("column-3", it->second[3].c_str());
-  ++it;
-  EXPECT_STREQ("family-2", it->first.c_str());
-  EXPECT_EQ(1, it->second.size());
-  EXPECT_STREQ("column-X", it->second[0].c_str());
-  ++it;
-  EXPECT_STREQ("family-3", it->first.c_str());
-  EXPECT_EQ(0, it->second.size());
-  ++it;
-  EXPECT_EQ(it, get.FamilyMap().end());
-}
-
-void CheckFamiliesAfterCopy(Get &get) {
-  EXPECT_EQ(true, get.HasFamilies());
-  EXPECT_EQ(3, get.FamilyMap().size());
-  int i = 1;
-  for (const auto &family : get.FamilyMap()) {
-    std::string family_name = "family-" + std::to_string(i);
-    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
-    i += 1;
-  }
-  // Check if the alreaday added CF's and CQ's are as expected
-  auto it = get.FamilyMap().begin();
-  EXPECT_STREQ("family-1", it->first.c_str());
-  EXPECT_EQ(4, it->second.size());
-  EXPECT_STREQ("column-1", it->second[0].c_str());
-  EXPECT_STREQ("column-2", it->second[1].c_str());
-  EXPECT_STREQ("", it->second[2].c_str());
-  EXPECT_STREQ("column-3", it->second[3].c_str());
-  ++it;
-  EXPECT_STREQ("family-2", it->first.c_str());
-  EXPECT_EQ(1, it->second.size());
-  EXPECT_STREQ("column-X", it->second[0].c_str());
-  ++it;
-  EXPECT_STREQ("family-3", it->first.c_str());
-  EXPECT_EQ(0, it->second.size());
-  ++it;
-  EXPECT_EQ(it, get.FamilyMap().end());
-}
-
-void GetMethods(Get &get, const std::string &row) {
-  EXPECT_EQ(row, get.row());
-
-  CheckFamilies(get);
-  EXPECT_EQ(true, get.CacheBlocks());
-  get.SetCacheBlocks(false);
-  EXPECT_EQ(false, get.CacheBlocks());
-
-  EXPECT_EQ(hbase::pb::Consistency::STRONG, get.Consistency());
-  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
-  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, get.Consistency());
-
-  EXPECT_EQ(1, get.MaxVersions());
-  get.SetMaxVersions(2);
-  EXPECT_EQ(2, get.MaxVersions());
-  get.SetMaxVersions();
-  EXPECT_EQ(1, get.MaxVersions());
-
-  // Test initial values
-  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
-  EXPECT_EQ(std::numeric_limits<int64_t>::max(), get.Timerange().MaxTimeStamp());
-
-  // Set & Test new values using TimeRange and TimeStamp
-  get.SetTimeRange(1000, 2000);
-  EXPECT_EQ(1000, get.Timerange().MinTimeStamp());
-  EXPECT_EQ(2000, get.Timerange().MaxTimeStamp());
-  get.SetTimeStamp(0);
-  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
-  EXPECT_EQ(1, get.Timerange().MaxTimeStamp());
-
-  // Test some exceptions
-  ASSERT_THROW(get.SetTimeRange(-1000, 2000), std::runtime_error);
-  ASSERT_THROW(get.SetTimeRange(1000, -2000), std::runtime_error);
-  ASSERT_THROW(get.SetTimeRange(1000, 200), std::runtime_error);
-  ASSERT_THROW(get.SetTimeStamp(std::numeric_limits<int64_t>::max()), std::runtime_error);
-
-  // Test some exceptions
-  ASSERT_THROW(get.SetMaxVersions(0), std::runtime_error);
-  ASSERT_THROW(get.SetMaxVersions(std::numeric_limits<uint32_t>::max() + 1), std::runtime_error);
-}
-
-TEST(Get, SingleGet) {
-  std::string row_str = "row-test";
-  Get get(row_str);
-  GetMethods(get, row_str);
-
-  Get get_tmp(row_str);
-  Get getcp(get_tmp);
-  GetMethods(getcp, row_str);
-
-  Get geteq("test");
-  geteq = get_tmp;
-  GetMethods(geteq, row_str);
-
-  // Adding the below tests as there were some concerns raised that the same
-  // vector of qualifiers in FamilyMap is being shared between copied objects
-  // Verify the source object's family map size before using it to copy.
-  EXPECT_EQ(3, get.FamilyMap().size());
-
-  Get getcp_fam(get);
-  // address of family maps should be different.
-  EXPECT_NE(&(get.FamilyMap()), &(getcp_fam.FamilyMap()));
-
-  // Add family to the source object
-  get.AddColumn("family-4", "column-A");
-  get.AddColumn("family-4", "column-B");
-  // Verify the source object's family map size
-  EXPECT_EQ(4, get.FamilyMap().size());
-  // Verify the source object's family elements
-  auto it = get.FamilyMap().begin();
-  EXPECT_STREQ("family-1", it->first.c_str());
-  EXPECT_EQ(4, it->second.size());
-  EXPECT_STREQ("column-1", it->second[0].c_str());
-  EXPECT_STREQ("column-2", it->second[1].c_str());
-  EXPECT_STREQ("", it->second[2].c_str());
-  EXPECT_STREQ("column-3", it->second[3].c_str());
-  ++it;
-  EXPECT_STREQ("family-2", it->first.c_str());
-  EXPECT_EQ(1, it->second.size());
-  EXPECT_STREQ("column-X", it->second[0].c_str());
-  ++it;
-  EXPECT_STREQ("family-3", it->first.c_str());
-  EXPECT_EQ(0, it->second.size());
-  ++it;
-  EXPECT_STREQ("family-4", it->first.c_str());
-  EXPECT_EQ(2, it->second.size());
-  EXPECT_STREQ("column-A", it->second[0].c_str());
-  EXPECT_STREQ("column-B", it->second[1].c_str());
-  ++it;
-  EXPECT_EQ(it, get.FamilyMap().end());
-
-  // Verifying the copied object's families. It will remain unchanged and below
-  // tests should pass
-  CheckFamiliesAfterCopy(getcp_fam);
-}
-
-TEST(Get, MultiGet) {
-  std::vector<std::unique_ptr<Get>> gets;
-  for (int i = 0; i < NUMBER_OF_GETS; i++) {
-    std::string row_str = "row-test";
-    row_str += std::to_string(i);
-    auto get = std::make_unique<Get>(row_str);
-
-    GetMethods(*get, row_str);
-    gets.push_back(std::move(get));
-  }
-  EXPECT_EQ(NUMBER_OF_GETS, gets.size());
-}
-
-TEST(Get, Exception) {
-  std::string row(std::numeric_limits<int16_t>::max() + 1, 'X');
-  ASSERT_THROW(Get tmp = Get(row), std::runtime_error);
-  ASSERT_THROW(Get tmp = Get(""), std::runtime_error);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/get.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.cc b/hbase-native-client/core/get.cc
deleted file mode 100644
index bc0d446..0000000
--- a/hbase-native-client/core/get.cc
+++ /dev/null
@@ -1,123 +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/get.h"
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-
-namespace hbase {
-
-Get::~Get() {}
-
-Get::Get(const std::string &row) : Row(row) {}
-
-Get::Get(const Get &get) : Query(get) {
-  row_ = get.row_;
-  max_versions_ = get.max_versions_;
-  cache_blocks_ = get.cache_blocks_;
-  check_existence_only_ = get.check_existence_only_;
-  consistency_ = get.consistency_;
-  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
-  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
-}
-
-Get &Get::operator=(const Get &get) {
-  Query::operator=(get);
-  row_ = get.row_;
-  max_versions_ = get.max_versions_;
-  cache_blocks_ = get.cache_blocks_;
-  check_existence_only_ = get.check_existence_only_;
-  consistency_ = get.consistency_;
-  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
-  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
-  return *this;
-}
-
-Get &Get::AddFamily(const std::string &family) {
-  const auto &it = family_map_.find(family);
-
-  /**
-   * Check if any qualifiers are already present or not.
-   * Remove all existing qualifiers if the given family is already present in
-   * the map
-   */
-  if (family_map_.end() != it) {
-    it->second.clear();
-  } else {
-    family_map_[family];
-  }
-  return *this;
-}
-
-Get &Get::AddColumn(const std::string &family, const std::string &qualifier) {
-  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
-
-  /**
-   * Check if any qualifiers are already present or not.
-   * Add only if qualifiers for a given family are not present
-   */
-  if (it == family_map_[family].end()) {
-    family_map_[family].push_back(qualifier);
-  }
-  return *this;
-}
-
-hbase::pb::Consistency Get::Consistency() const { return consistency_; }
-
-Get &Get::SetConsistency(hbase::pb::Consistency consistency) {
-  consistency_ = consistency;
-  return *this;
-}
-
-bool Get::HasFamilies() const { return !family_map_.empty(); }
-
-const std::map<std::string, std::vector<std::string>> &Get::FamilyMap() const {
-  return family_map_;
-}
-
-int Get::MaxVersions() const { return max_versions_; }
-
-Get &Get::SetMaxVersions(int32_t max_versions) {
-  if (0 == max_versions) throw std::runtime_error("max_versions must be positive");
-
-  max_versions_ = max_versions;
-  return *this;
-}
-
-bool Get::CacheBlocks() const { return cache_blocks_; }
-
-Get &Get::SetCacheBlocks(bool cache_blocks) {
-  cache_blocks_ = cache_blocks;
-  return *this;
-}
-
-Get &Get::SetTimeRange(int64_t min_timestamp, int64_t max_timestamp) {
-  tr_.reset(new TimeRange(min_timestamp, max_timestamp));
-  return *this;
-}
-
-Get &Get::SetTimeStamp(int64_t timestamp) {
-  tr_.reset(new TimeRange(timestamp, timestamp + 1));
-  return *this;
-}
-
-const TimeRange &Get::Timerange() const { return *tr_; }
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/get.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.h b/hbase-native-client/core/get.h
deleted file mode 100644
index c4cddfb..0000000
--- a/hbase-native-client/core/get.h
+++ /dev/null
@@ -1,132 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/query.h"
-#include "core/row.h"
-#include "core/time-range.h"
-#include "if/Client.pb.h"
-
-namespace hbase {
-
-class Get : public Row, public Query {
- public:
-  /**
-   * Constructors
-   */
-  explicit Get(const std::string& row);
-  Get(const Get& cget);
-  Get& operator=(const Get& cget);
-
-  ~Get();
-
-  /**
-   * @brief Returns the maximum number of values to fetch per CF
-   */
-  int MaxVersions() const;
-
-  /**
-   * @brief Get up to the specified number of versions of each column. default
-   * is 1.
-   * @param max_versions max_versons to set
-   */
-  Get& SetMaxVersions(int32_t max_versions = 1);
-
-  /**
-   * @brief Returns whether blocks should be cached for this Get operation.
-   */
-  bool CacheBlocks() const;
-
-  /**
-   * @brief Set whether blocks should be cached for this Get operation.
-   * @param cache_blocks to set
-   */
-  Get& SetCacheBlocks(bool cache_blocks);
-
-  /**
-   * @brief Returns the Get family map for this Get operation. Used
-   * for constructing Scan object with an already constructed Get
-   */
-  const std::map<std::string, std::vector<std::string>>& FamilyMap() const;
-
-  /**
-   * @brief Returns the timerange for this Get
-   */
-  const TimeRange& Timerange() const;
-
-  /**
-   * @brief Get versions of columns only within the specified timestamp range,
-   * [minStamp, maxStamp).
-   * @param minStamp the minimum timestamp, inclusive
-   * @param maxStamp the maximum timestamp, exclusive
-   */
-  Get& SetTimeRange(int64_t min_timestamp, int64_t max_timestamp);
-
-  /**
-   * @brief Get versions of columns with the specified timestamp.
-   * @param The timestamp to be set
-   */
-  Get& SetTimeStamp(int64_t timestamp);
-
-  /**
-   * @brief Get all columns from the specified family.
-   * @param family to be retrieved
-   */
-  Get& AddFamily(const std::string& family);
-
-  /**
-   *  @brief Get the column from the specific family with the specified
-   * qualifier.
-   *  @param family to be retrieved
-   *  @param qualifier to be retrieved
-   */
-  Get& AddColumn(const std::string& family, const std::string& qualifier);
-
-  /**
-   * @brief Returns true if family map is non empty false otherwise
-   */
-  bool HasFamilies() const;
-
-  /**
-   * @brief Returns the consistency level for this Get operation
-   */
-  hbase::pb::Consistency Consistency() const;
-
-  /**
-   * @brief Sets the consistency level for this Get operation
-   * @param Consistency to be set
-   */
-  Get& SetConsistency(hbase::pb::Consistency consistency);
-
- private:
-  int32_t max_versions_ = 1;
-  bool cache_blocks_ = true;
-  bool check_existence_only_ = false;
-  std::map<std::string, std::vector<std::string>> family_map_;
-  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
-  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/hbase-configuration-loader.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/hbase-configuration-loader.cc b/hbase-native-client/core/hbase-configuration-loader.cc
deleted file mode 100644
index 4d6a3e9..0000000
--- a/hbase-native-client/core/hbase-configuration-loader.cc
+++ /dev/null
@@ -1,208 +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/hbase-configuration-loader.h"
-
-#include <glog/logging.h>
-#include <boost/foreach.hpp>
-#include <boost/property_tree/ptree.hpp>
-#include <boost/property_tree/xml_parser.hpp>
-
-namespace hbase {
-
-bool is_valid_bool(const std::string &raw) {
-  if (raw.empty()) {
-    return false;
-  }
-
-  if (!strcasecmp(raw.c_str(), "true")) {
-    return true;
-  }
-  if (!strcasecmp(raw.c_str(), "false")) {
-    return true;
-  }
-  return false;
-}
-
-bool str_to_bool(const std::string &raw) {
-  if (!strcasecmp(raw.c_str(), "true")) {
-    return true;
-  }
-  return false;
-}
-
-HBaseConfigurationLoader::HBaseConfigurationLoader() {}
-
-HBaseConfigurationLoader::~HBaseConfigurationLoader() {}
-
-void HBaseConfigurationLoader::SetDefaultSearchPath() {
-  /*
-   * Try (in order, taking the first valid one):
-   * $HBASE_CONF_DIR
-   * /etc/hbase/conf
-   *
-   */
-  const char *hadoop_conf_dir_env = getenv("HBASE_CONF");
-  if (hadoop_conf_dir_env) {
-    AddToSearchPath(hadoop_conf_dir_env);
-  } else {
-    AddToSearchPath(kHBaseDefauktConfPath);
-  }
-}
-
-void HBaseConfigurationLoader::ClearSearchPath() { search_paths_.clear(); }
-
-void HBaseConfigurationLoader::SetSearchPath(const std::string &search_path) {
-  search_paths_.clear();
-
-  std::vector<std::string> paths;
-  std::string::size_type start = 0;
-  std::string::size_type end = search_path.find(kSearchPathSeparator);
-
-  while (end != std::string::npos) {
-    paths.push_back(search_path.substr(start, end - start));
-    start = ++end;
-    end = search_path.find(kSearchPathSeparator, start);
-  }
-  paths.push_back(search_path.substr(start, search_path.length()));
-
-  for (auto path : paths) {
-    AddToSearchPath(path);
-  }
-}
-
-void HBaseConfigurationLoader::AddToSearchPath(const std::string &search_path) {
-  if (search_path.empty()) return;
-
-  std::string path_to_add(search_path);
-  if (search_path.back() != kFileSeparator) {
-    path_to_add += kFileSeparator;
-  }
-  if (std::find(search_paths_.begin(), search_paths_.end(), path_to_add) == search_paths_.end())
-    search_paths_.push_back(path_to_add);
-}
-
-void HBaseConfigurationLoader::AddDefaultResources() {
-  resources_.push_back(kHBaseDefaultXml);
-  resources_.push_back(kHBaseSiteXml);
-}
-
-void HBaseConfigurationLoader::AddResources(const std::string &filename) {
-  if (std::find(resources_.begin(), resources_.end(), filename) == resources_.end())
-    resources_.push_back(filename);
-}
-
-optional<Configuration> HBaseConfigurationLoader::LoadDefaultResources() {
-  SetDefaultSearchPath();
-  AddDefaultResources();
-  ConfigMap conf_property;
-  bool success = false;
-  for (auto dir : search_paths_) {
-    for (auto file : resources_) {
-      std::string config_file = dir + file;
-      std::ifstream stream(config_file);
-      if (stream.is_open()) {
-        success |= LoadProperties(config_file, conf_property);
-      } else {
-        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
-      }
-    }
-  }
-  if (success) {
-    return std::experimental::make_optional<Configuration>(Configuration(conf_property));
-  } else {
-    return optional<Configuration>();
-  }
-}
-
-optional<Configuration> HBaseConfigurationLoader::LoadResources(
-    const std::string &search_path, const std::vector<std::string> &resources) {
-  SetSearchPath(search_path);
-  for (const auto &resource : resources) AddResources(resource);
-  ConfigMap conf_property;
-  bool success = false;
-  for (auto dir : search_paths_) {
-    for (auto file : resources_) {
-      std::string config_file = dir + file;
-      std::ifstream stream(config_file);
-      if (stream.is_open()) {
-        success |= LoadProperties(config_file, conf_property);
-      } else {
-        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
-      }
-    }
-  }
-  if (success) {
-    return std::experimental::make_optional<Configuration>(Configuration(conf_property));
-  } else {
-    return optional<Configuration>();
-  }
-}
-
-bool HBaseConfigurationLoader::LoadProperties(const std::string &file, ConfigMap &property_map) {
-  // Create empty property tree object
-  using boost::property_tree::ptree;
-  ptree pt;
-  try {
-    // Load XML file and put contents in a property tree.
-    // If read fails, throw exception.
-    read_xml(file, pt);
-
-    // If configuration key is not found exception is thrown
-    std::string configuration = pt.get<std::string>("configuration");
-
-    // Iterate over configuration section.
-    // Store all found properties in ConfigMap
-    BOOST_FOREACH (ptree::value_type &v, pt.get_child("configuration")) {
-      if ("property" == v.first) {
-        std::string name_node = v.second.get<std::string>("name");
-        std::string value_node = v.second.get<std::string>("value");
-        if ((name_node.size() > 0) && (value_node.size() > 0)) {
-          boost::optional<std::string> final_node = v.second.get_optional<std::string>("final");
-          UpdateMapWithValue(property_map, name_node, value_node, final_node);
-        }
-      }
-    }
-  } catch (std::exception &ex) {
-    DLOG(WARNING) << "Exception in parsing file [" << file << "]:[" << ex.what() << "]";
-    return false;
-  }
-  return true;
-}
-
-bool HBaseConfigurationLoader::UpdateMapWithValue(ConfigMap &map, const std::string &key,
-                                                  const std::string &value,
-                                                  boost::optional<std::string> final_text) {
-  auto map_value = map.find(key);
-  if (map_value != map.end() && map_value->second.final) {
-    return false;
-  }
-
-  bool final_value = false;
-  if (nullptr != final_text.get_ptr()) {
-    if (is_valid_bool(final_text.get())) {
-      final_value = str_to_bool(final_text.get());
-    }
-  }
-
-  map[key].value = value;
-  map[key].final = final_value;
-  return true;
-}
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/hbase-configuration-loader.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/hbase-configuration-loader.h b/hbase-native-client/core/hbase-configuration-loader.h
deleted file mode 100644
index 95b2541..0000000
--- a/hbase-native-client/core/hbase-configuration-loader.h
+++ /dev/null
@@ -1,140 +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.
- *
- */
-
-#pragma once
-
-#include <map>
-#include <string>
-#include <vector>
-
-#include <boost/optional.hpp>
-
-#include "core/configuration.h"
-#include "utils/optional.h"
-
-namespace hbase {
-
-class HBaseConfigurationLoader {
- public:
-  HBaseConfigurationLoader();
-  ~HBaseConfigurationLoader();
-
-  /**
-   * @brief Creates a Configuration object based on default resources loaded
-   * from default search paths. Default search path will be either $HBASE_CONF
-   * is set or /etc/hbase/conf. Default resources are hbase-default.xml and
-   * hbase-site.xml.SetDefaultSearchPath() and AddDefaultResources() are used
-   * for the same.
-   * Values are loaded in from hbase-default.xml first and then from
-   * hbase-site.xml.
-   * Properties in hbase-site.xml will override the ones in hbase-default.xml
-   * unless marked as final
-   */
-  optional<Configuration> LoadDefaultResources();
-
-  /*
-   * @brief Creates a Configuration object based on resources loaded from search
-   * paths. Search paths are defined in search_path. Values are loaded from
-   * resources and will be overridden unless marked as final
-   * @param search_path - ':' search paths to load resources.
-   * @param resources - list of resources used to load configuration properties.
-   */
-  optional<Configuration> LoadResources(const std::string &search_path,
-                                        const std::vector<std::string> &resources);
-
- private:
-  using ConfigMap = Configuration::ConfigMap;
-  const std::string kHBaseDefaultXml = "hbase-default.xml";
-  const std::string kHBaseSiteXml = "hbase-site.xml";
-  const std::string kHBaseDefauktConfPath = "/etc/hbase/conf";
-
-  // Adds FILE_SEPARATOR to the search path
-  const char kFileSeparator = '/';
-
-  // Separator using which multiple search paths can be defined.
-  const char kSearchPathSeparator = ':';
-
-  /**
-   * List of paths which will be looked up for loading properties.
-   */
-  std::vector<std::string> search_paths_;
-
-  /**
-   * List of files which will be looked up in search_paths_ to load properties.
-   */
-  std::vector<std::string> resources_;
-
-  /**
-   * @brief This method sets the search path to the default search path (i.e.
-   * "$HBASE_CONF" or "/etc/hbase/conf" if HBASE_CONF is absent)
-   */
-  void SetDefaultSearchPath();
-
-  /**
-   * @brief Clears out the set search path(s)
-   */
-  void ClearSearchPath();
-
-  /**
-   * @brief Sets the search path to ":"-delimited paths, clearing already
-   * defined values
-   * @param search_path Single path or ":"-delimited separated paths
-   */
-  void SetSearchPath(const std::string &search_path);
-
-  /**
-   * @brief Adds an element to the search path if not already present.
-   * @param search_path Path that will be added to load config values
-   */
-  void AddToSearchPath(const std::string &search_path);
-
-  /**
-   * @brief This method will add default resources i.e. hbase-default.xml and
-   * hbase-site.xml to the default search path.
-   */
-  void AddDefaultResources();
-
-  /**
-   * @brief Adds resources to list for loading config values.
-   * @param filename to be added to resources.
-   */
-  void AddResources(const std::string &filename);
-
-  /**
-   * @brief Loads properties in file identified by file in a map identified by
-   * property_map
-   * @param file XML file which defines HBase configuration properties.
-   * @param property_map Property map representing HBase configuration as key
-   * value pairs.
-   * @throws Boost ptree exception if some parsing issue occurs.
-   */
-  bool LoadProperties(const std::string &file, ConfigMap &property_map);
-
-  /**
-   * @brief This method will create a map of the name and properties.
-   * @param map Property map to hold configuration properties.
-   * @param key value of name node.
-   * @param value value of value node.
-   * @param final_text value of final node true or false if present
-   */
-  bool UpdateMapWithValue(ConfigMap &map, const std::string &key, const std::string &value,
-                          boost::optional<std::string> final_text);
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/hbase-configuration-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/hbase-configuration-test.cc b/hbase-native-client/core/hbase-configuration-test.cc
deleted file mode 100644
index 56b4463..0000000
--- a/hbase-native-client/core/hbase-configuration-test.cc
+++ /dev/null
@@ -1,375 +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 <fstream>
-#include <iostream>
-
-#include <glog/logging.h>
-#include <gtest/gtest.h>
-#include <boost/filesystem.hpp>
-#include "core/configuration.h"
-#include "core/hbase-configuration-loader.h"
-
-using namespace hbase;
-using std::experimental::nullopt;
-
-const std::string kDefHBaseConfPath("./build/test-data/hbase-configuration-test/conf/");
-const std::string kHBaseConfPath("./build/test-data/hbase-configuration-test/custom-conf/");
-
-const std::string kHBaseDefaultXml("hbase-default.xml");
-const std::string kHBaseSiteXml("hbase-site.xml");
-
-const std::string kHBaseDefaultXmlData(
-    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
-    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
-    "Software Foundation (ASF) under one\n * or more contributor license "
-    "agreements.  See the NOTICE file\n * distributed with this work for "
-    "additional information\n * regarding copyright ownership.  The ASF "
-    "licenses this file\n * to you under the Apache License, Version 2.0 "
-    "(the\n * \"License\"); you may not use this file except in compliance\n * "
-    "with the License.  You may obtain a copy of the License at\n *\n *     "
-    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
-    "applicable law or agreed to in writing, software\n * distributed under "
-    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
-    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
-    "for the specific language governing permissions and\n * limitations under "
-    "the License.\n "
-    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
-    "name>\n<value>/root/hbase-docker/apps/hbase/data</value>\n<final>true</"
-    "final>\n</"
-    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
-    "name>\n<value>This value will be "
-    "overwritten</value>\n<final>false</final>\n</"
-    "property>\n\n<property>\n<name>default-prop</name>\n<value>default-value</"
-    "value>\n</property>\n\n</configuration>");
-const std::string kHBaseSiteXmlData(
-    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
-    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
-    "Software Foundation (ASF) under one\n * or more contributor license "
-    "agreements.  See the NOTICE file\n * distributed with this work for "
-    "additional information\n * regarding copyright ownership.  The ASF "
-    "licenses this file\n * to you under the Apache License, Version 2.0 "
-    "(the\n * \"License\"); you may not use this file except in compliance\n * "
-    "with the License.  You may obtain a copy of the License at\n *\n *     "
-    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
-    "applicable law or agreed to in writing, software\n * distributed under "
-    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
-    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
-    "for the specific language governing permissions and\n * limitations under "
-    "the License.\n "
-    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
-    "name>\n<value>This value will not be be "
-    "overwritten</value>\n</"
-    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
-    "name>\n<value>/root/hbase-docker/zookeeper</value>\n</"
-    "property>\n\n<property>\n<name>hbase-client.user.name</"
-    "name>\n<value>${user.name}</value>\n</"
-    "property>\n\n<property>\n<name>hbase-client.user.dir</"
-    "name>\n<value>${user.dir}</value>\n</"
-    "property>\n\n<property>\n<name>hbase-client.user.home</"
-    "name>\n<value>${user.home}</value>\n</"
-    "property>\n\n<property>\n<name>selfRef</name>\n<value>${selfRef}</"
-    "value>\n</property>\n\n<property>\n<name>foo.substs</"
-    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
-    "bar},${bar},</value>\n</"
-    "property>\n\n<property>\n<name>foo.substs.exception</"
-    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
-    "bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar}"
-    ",${bar},${bar}</value>\n</property>\n\n<property>\n<name>bar</"
-    "name>\n<value>bar-value</value>\n</"
-    "property>\n\n<property>\n<name>custom-prop</name>\n<value>custom-value</"
-    "value>\n</property>\n\n<property>\n<name>int</name>\n<value>16000</"
-    "value>\n</property>\n\n<property>\n<name>int.largevalue</"
-    "name>\n<value>2147483646</value>\n</"
-    "property>\n\n<property>\n<name>int.exception</name>\n<value>2147483648</"
-    "value>\n</property>\n\n<property>\n<name>long</name>\n<value>2147483850</"
-    "value>\n</property>\n\n<property>\n<name>long.largevalue</"
-    "name>\n<value>9223372036854775807</value>\n</"
-    "property>\n\n<property>\n<name>long.exception</"
-    "name>\n<value>9223372036854775810</value>\n</"
-    "property>\n\n<property>\n<name>double</name>\n<value>17.9769e+100</"
-    "value>\n</property>\n\n<property>\n<name>double.largevalue</"
-    "name>\n<value>170.769e+200</value>\n</"
-    "property>\n\n<property>\n<name>double.exception</"
-    "name>\n<value>1.79769e+310</value>\n</"
-    "property>\n\n<property>\n<name>bool.true</name>\n<value>true</value>\n</"
-    "property>\n\n<property>\n<name>bool.false</name>\n<value>false</value>\n</"
-    "property>\n\n<property>\n<name>bool.exception</name>\n<value>unknown "
-    "bool</value>\n</property>\n\n</configuration>");
-
-void WriteDataToFile(const std::string &file, const std::string &xml_data) {
-  std::ofstream hbase_conf;
-  hbase_conf.open(file.c_str());
-  hbase_conf << xml_data;
-  hbase_conf.close();
-}
-
-void CreateHBaseConf(const std::string &dir, const std::string &file, const std::string xml_data) {
-  // Directory will be created if not present
-  if (!boost::filesystem::exists(dir)) {
-    boost::filesystem::create_directories(dir);
-  }
-  // Remove temp file always
-  boost::filesystem::remove((dir + file).c_str());
-  WriteDataToFile((dir + file), xml_data);
-}
-
-void CreateHBaseConfWithEnv() {
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
-  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 1);
-}
-
-/*
- * Config will be loaded from $HBASE_CONF. We set it @ kDefHBaseConfPath
- * Config values will be loaded from hbase-default.xml and hbase-site.xml
- * present in the above path.
- */
-TEST(Configuration, LoadConfFromDefaultLocation) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
-  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 0);
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value").c_str(), "custom-value");
-  EXPECT_STREQ((*conf).Get("default-prop", "Set this value").c_str(), "default-value");
-}
-
-/*
- * Config will be loaded from hbase-site.xml defined at
- * kHBaseConfPath
- */
-TEST(Configuration, LoadConfFromCustomLocation) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
-
-  HBaseConfigurationLoader loader;
-  std::vector<std::string> resources{kHBaseSiteXml};
-  hbase::optional<Configuration> conf = loader.LoadResources(kHBaseConfPath, resources);
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
-  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
-}
-
-/*
- * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
- * kDefHBaseConfPath and kHBaseConfPath respectively.
- */
-TEST(Configuration, LoadConfFromMultipleLocatons) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
-  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
-  CreateHBaseConf(kHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
-  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
-
-  HBaseConfigurationLoader loader;
-  std::string conf_paths = kDefHBaseConfPath + ":" + kHBaseConfPath;
-  std::vector<std::string> resources{kHBaseDefaultXml, kHBaseSiteXml};
-  hbase::optional<Configuration> conf = loader.LoadResources(conf_paths, resources);
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("default-prop", "From hbase-default.xml").c_str(), "default-value");
-  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
-  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
-}
-
-/*
- * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
- * $HBASE_CONF.
- * We set HBASE_CONF to kDefHBaseConfPath
- * Below tests load the conf files in the same way unless specified.
- */
-TEST(Configuration, DefaultValues) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("default-prop", "Set this value.").c_str(), "default-value");
-  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value.").c_str(), "custom-value");
-}
-
-TEST(Configuration, FinalValues) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("hbase.rootdir", "").c_str(), "/root/hbase-docker/apps/hbase/data");
-  EXPECT_STREQ((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
-               "/root/hbase-docker/zookeeper");
-  EXPECT_STRNE((*conf).Get("hbase.rootdir", "").c_str(), "This value will not be be overwritten");
-  EXPECT_STRNE((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
-               "This value will be overwritten");
-}
-
-/*
- * Config will be loaded from HBASE_CONF which we set in
- * CreateHBaseConfWithEnv().
- * Config values will be loaded from hbase-default.xml and hbase-site.xml in the
- * above path.
- */
-TEST(Configuration, EnvVars) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("hbase-client.user.name", "").c_str(), "${user.name}");
-  EXPECT_STRNE((*conf).Get("hbase-client.user.name", "root").c_str(), "test-user");
-}
-
-TEST(Configuration, SelfRef) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("selfRef", "${selfRef}").c_str(), "${selfRef}");
-}
-
-TEST(Configuration, VarExpansion) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_STREQ((*conf).Get("foo.substs", "foo-value").c_str(),
-               "bar-value,bar-value,bar-value,bar-value,bar-value,bar-value,"
-               "bar-value,bar-value,bar-value,bar-value,");
-  EXPECT_STRNE((*conf).Get("foo.substs", "foo-value").c_str(), "bar-value");
-}
-
-TEST(Configuration, VarExpansionException) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  ASSERT_THROW((*conf).Get("foo.substs.exception", "foo-value").c_str(), std::runtime_error);
-}
-
-TEST(Configuration, GetInt) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_EQ(16000, (*conf).GetInt("int", 0));
-  EXPECT_EQ(2147483646, (*conf).GetInt("int.largevalue", 0));
-}
-
-TEST(Configuration, GetLong) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_EQ(2147483850, (*conf).GetLong("long", 0));
-  EXPECT_EQ(9223372036854775807, (*conf).GetLong("long.largevalue", 0));
-}
-
-TEST(Configuration, GetDouble) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_DOUBLE_EQ(17.9769e+100, (*conf).GetDouble("double", 0.0));
-  EXPECT_DOUBLE_EQ(170.769e+200, (*conf).GetDouble("double.largevalue", 0.0));
-}
-
-TEST(Configuration, GetBool) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  EXPECT_EQ(true, (*conf).GetBool("bool.true", true));
-  EXPECT_EQ(false, (*conf).GetBool("bool.false", false));
-}
-
-TEST(Configuration, GetIntException) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  ASSERT_THROW((*conf).GetInt("int.exception", 0), std::runtime_error);
-}
-
-TEST(Configuration, GetLongException) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  ASSERT_THROW((*conf).GetLong("long.exception", 0), std::runtime_error);
-}
-
-TEST(Configuration, GetDoubleException) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  ASSERT_THROW((*conf).GetDouble("double.exception", 0), std::runtime_error);
-}
-
-TEST(Configuration, GetBoolException) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  CreateHBaseConfWithEnv();
-
-  HBaseConfigurationLoader loader;
-  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
-  ASSERT_TRUE(conf != nullopt) << "No configuration object present.";
-  ASSERT_THROW((*conf).GetBool("bool.exception", false), std::runtime_error);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/hbase-rpc-controller.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/hbase-rpc-controller.cc b/hbase-native-client/core/hbase-rpc-controller.cc
deleted file mode 100644
index bc53781..0000000
--- a/hbase-native-client/core/hbase-rpc-controller.cc
+++ /dev/null
@@ -1,22 +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/hbase-rpc-controller.h"
-
-namespace hbase {} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/hbase-rpc-controller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/hbase-rpc-controller.h b/hbase-native-client/core/hbase-rpc-controller.h
deleted file mode 100644
index 33f552b..0000000
--- a/hbase-native-client/core/hbase-rpc-controller.h
+++ /dev/null
@@ -1,59 +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.
- *
- */
-#pragma once
-
-#include <folly/ExceptionWrapper.h>
-#include <google/protobuf/service.h>
-#include <chrono>
-#include <string>
-
-namespace hbase {
-
-class HBaseRpcController : public google::protobuf::RpcController {
- public:
-  HBaseRpcController() {}
-  virtual ~HBaseRpcController() = default;
-
-  void set_call_timeout(const std::chrono::milliseconds& call_timeout) {
-    // TODO:
-  }
-
-  void Reset() override {}
-
-  bool Failed() const override { return false; }
-
-  folly::exception_wrapper exception() { return exception_; }
-
-  void set_exception(const folly::exception_wrapper& exception) { exception_ = exception; }
-
-  std::string ErrorText() const override { return ""; }
-
-  void StartCancel() override {}
-
-  void SetFailed(const std::string& reason) override {}
-
-  bool IsCanceled() const override { return false; }
-
-  void NotifyOnCancel(google::protobuf::Closure* callback) override {}
-
- private:
-  folly::exception_wrapper exception_;
-};
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/increment-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/increment-test.cc b/hbase-native-client/core/increment-test.cc
deleted file mode 100644
index aa9b864..0000000
--- a/hbase-native-client/core/increment-test.cc
+++ /dev/null
@@ -1,129 +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 <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include "core/increment.h"
-#include "core/mutation.h"
-#include "core/put.h"
-#include "utils/time-util.h"
-
-using hbase::Increment;
-using hbase::Increment;
-using hbase::Cell;
-using hbase::CellType;
-using hbase::Mutation;
-using hbase::TimeUtil;
-
-const constexpr int64_t Mutation::kLatestTimestamp;
-
-TEST(Increment, Row) {
-  Increment incr{"foo"};
-  EXPECT_EQ("foo", incr.row());
-}
-
-TEST(Increment, Durability) {
-  Increment incr{"row"};
-  EXPECT_EQ(hbase::pb::MutationProto_Durability_USE_DEFAULT, incr.Durability());
-
-  auto skipWal = hbase::pb::MutationProto_Durability_SKIP_WAL;
-  incr.SetDurability(skipWal);
-  EXPECT_EQ(skipWal, incr.Durability());
-}
-
-TEST(Increment, Timestamp) {
-  Increment incr{"row"};
-
-  // test default timestamp
-  EXPECT_EQ(Mutation::kLatestTimestamp, incr.TimeStamp());
-
-  // set custom timestamp
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  incr.SetTimeStamp(ts);
-  EXPECT_EQ(ts, incr.TimeStamp());
-
-  // Add a column with custom timestamp
-  incr.AddColumn("f", "q", 5l);
-  auto &cell = incr.FamilyMap().at("f")[0];
-  EXPECT_EQ(ts, cell->Timestamp());
-}
-
-TEST(Increment, HasFamilies) {
-  Increment incr{"row"};
-
-  EXPECT_EQ(false, incr.HasFamilies());
-
-  incr.AddColumn("f", "q", 5l);
-  EXPECT_EQ(true, incr.HasFamilies());
-}
-
-TEST(Increment, Add) {
-  CellType cell_type = CellType::PUT;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-
-  // add first cell
-  Increment incr{"row"};
-  incr.Add(std::move(cell));
-  EXPECT_EQ(1, incr.FamilyMap().size());
-  EXPECT_EQ(1, incr.FamilyMap().at(family).size());
-
-  // add a non-matching row
-  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, value, cell_type);
-  Increment incr2{"foo"};
-  ASSERT_THROW(incr2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
-
-  // add a second cell with same family
-  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, value, cell_type);
-  incr.Add(std::move(cell3));
-  EXPECT_EQ(1, incr.FamilyMap().size());
-  EXPECT_EQ(2, incr.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, value, cell_type);
-  incr.Add(std::move(cell4));
-  EXPECT_EQ(2, incr.FamilyMap().size());
-  EXPECT_EQ(1, incr.FamilyMap().at("family-2").size());
-}
-
-TEST(Increment, AddColumn) {
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
-
-  Increment incr{"row"};
-  incr.AddColumn(family, column, 5l);
-  EXPECT_EQ(1, incr.FamilyMap().size());
-  EXPECT_EQ(1, incr.FamilyMap().at(family).size());
-
-  // add a second cell with same family
-  incr.AddColumn(family, "column-2", 6l);
-  EXPECT_EQ(1, incr.FamilyMap().size());
-  EXPECT_EQ(2, incr.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  incr.AddColumn("family-2", column, 7l);
-  EXPECT_EQ(2, incr.FamilyMap().size());
-  EXPECT_EQ(1, incr.FamilyMap().at("family-2").size());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/increment.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/increment.cc b/hbase-native-client/core/increment.cc
deleted file mode 100644
index a21a764..0000000
--- a/hbase-native-client/core/increment.cc
+++ /dev/null
@@ -1,56 +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/increment.h"
-#include <folly/Conv.h>
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-#include <utility>
-
-#include "utils/bytes-util.h"
-
-namespace hbase {
-
-/**
- *  @brief Increment the column from the specific family with the specified qualifier
- * by the specified amount.
- *  @param family family name
- *  @param qualifier column qualifier
- *  @param amount amount to increment by
- */
-Increment& Increment::AddColumn(const std::string& family, const std::string& qualifier,
-                                int64_t amount) {
-  family_map_[family].push_back(std::move(std::make_unique<Cell>(
-      row_, family, qualifier, timestamp_, BytesUtil::ToString(amount), hbase::CellType::PUT)));
-  return *this;
-}
-Increment& Increment::Add(std::unique_ptr<Cell> cell) {
-  if (cell->Row() != row_) {
-    throw std::runtime_error("The row in " + cell->DebugString() +
-                             " doesn't match the original one " + row_);
-  }
-
-  family_map_[cell->Family()].push_back(std::move(cell));
-  return *this;
-}
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/increment.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/increment.h b/hbase-native-client/core/increment.h
deleted file mode 100644
index 330f44a..0000000
--- a/hbase-native-client/core/increment.h
+++ /dev/null
@@ -1,57 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/cell.h"
-#include "core/mutation.h"
-
-namespace hbase {
-
-class Increment : public Mutation {
- public:
-  /**
-   * Constructors
-   */
-  explicit Increment(const std::string& row) : Mutation(row) {}
-  Increment(const Increment& cincrement) : Mutation(cincrement) {}
-  Increment& operator=(const Increment& cincrement) {
-    Mutation::operator=(cincrement);
-    return *this;
-  }
-
-  ~Increment() = default;
-
-  /**
-   *  @brief Increment the column from the specific family with the specified qualifier
-   * by the specified amount.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param amount amount to increment by
-   */
-  Increment& AddColumn(const std::string& family, const std::string& qualifier, int64_t amount);
-  Increment& Add(std::unique_ptr<Cell> cell);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/keyvalue-codec.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/keyvalue-codec.cc b/hbase-native-client/core/keyvalue-codec.cc
deleted file mode 100644
index 1b526b9..0000000
--- a/hbase-native-client/core/keyvalue-codec.cc
+++ /dev/null
@@ -1,69 +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/keyvalue-codec.h"
-
-#include <string>
-
-namespace hbase {
-
-KeyValueCodec::KVDecoder::KVDecoder(std::unique_ptr<folly::IOBuf> cell_block, uint32_t offset,
-                                    uint32_t length)
-    : cell_block_(std::move(cell_block)), offset_(offset), length_(length) {}
-
-KeyValueCodec::KVDecoder::~KVDecoder() {}
-
-std::shared_ptr<Cell> KeyValueCodec::KVDecoder::Decode(folly::io::Cursor &cursor) {
-  uint32_t key_length = cursor.readBE<uint32_t>();
-  uint32_t value_length = cursor.readBE<uint32_t>();
-  uint16_t row_length = cursor.readBE<uint16_t>();
-  std::string row = cursor.readFixedString(row_length);
-  uint8_t column_family_length = cursor.readBE<uint8_t>();
-  std::string column_family = cursor.readFixedString(column_family_length);
-  int qualifier_length =
-      key_length - (row_length + column_family_length + kHBaseSizeOfKeyInfrastructure_);
-  std::string column_qualifier = cursor.readFixedString(qualifier_length);
-  uint64_t timestamp = cursor.readBE<uint64_t>();
-  uint8_t key_type = cursor.readBE<uint8_t>();
-  std::string value = cursor.readFixedString(value_length);
-
-  return std::make_shared<Cell>(row, column_family, column_qualifier, timestamp, value,
-                                static_cast<hbase::CellType>(key_type));
-}
-
-bool KeyValueCodec::KVDecoder::Advance() {
-  if (end_of_cell_block_) {
-    return false;
-  }
-
-  if (cur_pos_ == length_) {
-    end_of_cell_block_ = true;
-    return false;
-  }
-
-  folly::io::Cursor cursor(cell_block_.get());
-  cursor.skip(offset_ + cur_pos_);
-  uint32_t current_cell_size = cursor.readBE<uint32_t>();
-  current_cell_ = Decode(cursor);
-  cur_pos_ += kHBaseSizeOfInt_ + current_cell_size;
-  return true;
-}
-
-uint32_t KeyValueCodec::KVDecoder::CellBlockLength() const { return length_; }
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/keyvalue-codec.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/keyvalue-codec.h b/hbase-native-client/core/keyvalue-codec.h
deleted file mode 100644
index fd58346..0000000
--- a/hbase-native-client/core/keyvalue-codec.h
+++ /dev/null
@@ -1,147 +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.
- *
- */
-
-#pragma once
-
-#include <folly/io/Cursor.h>
-#include <folly/io/IOBuf.h>
-#include <memory>
-
-#include "core/cell.h"
-#include "serde/codec.h"
-
-namespace hbase {
-
-/**
- * @brief Class for parsing sequence of Cells based on org.apache.hadoop.hbase.KeyValueCodec.java
- *
- * KeyValueCodec implements CellScanner interface. Sequence of cells are obtained from cell_block.
- * We have CreateEncoder and CreateDecoder public methods which will return Encoder/Decoder
- * instances which will be used to obtain individual cells in cell_block.
- * Usage:-
- * 1) Cell Decoding:-
- * unique_ptr<CellScanner> cell_scanner = KeyValueCodec::CreateDecoder(cell_block, cb_start_offset,
- *    cb_length);
- * while (cell_scanner->Advance()) {
- *  auto current_cell = cell_scanner->Current
- * }
- */
-class KeyValueCodec : public Codec {
- public:
-  /**
-        * Constructor
-        */
-  KeyValueCodec() {}
-
-  std::unique_ptr<Codec::Encoder> CreateEncoder() override { return std::make_unique<KVEncoder>(); }
-  std::unique_ptr<Codec::Decoder> CreateDecoder(std::unique_ptr<folly::IOBuf> cell_block,
-                                                uint32_t offset, uint32_t length) override {
-    return std::make_unique<KVDecoder>(std::move(cell_block), offset, length);
-  }
-
-  /** @brief returns the java class name corresponding to this Codec implementation */
-  virtual const char* java_class_name() const override { return kJavaClassName; }
-
-  static constexpr const char* kJavaClassName = "org.apache.hadoop.hbase.codec.KeyValueCodec";
-
- private:
-  class KVEncoder : public Codec::Encoder {
-   public:
-    KVEncoder() {}
-
-    void Write(const Cell& cell) {
-      // TODO: Encode Cells using KeyValueCodec wire format
-    }
-
-    void Flush() {}
-  };
-
-  class KVDecoder : public Codec::Decoder {
-   public:
-    KVDecoder(std::unique_ptr<folly::IOBuf> cell_block, uint32_t cell_block_start_offset,
-              uint32_t cell_block_length);
-    ~KVDecoder();
-
-    /**
-     * @brief Overridden from CellScanner. This method parses cell_block and stores the current in
-     * current_cell_. Current cell can be obtained using cell_scanner.Current();
-     */
-    bool Advance();
-
-    /**
-     * @brief returns the current cell
-     */
-    const std::shared_ptr<Cell> Current() const { return current_cell_; }
-
-    /**
-     * @brief returns the total length of cell_meta_block
-     */
-    uint32_t CellBlockLength() const;
-
-   private:
-    std::shared_ptr<Cell> Decode(folly::io::Cursor& cursor);
-
-    /**
-     * Size of boolean in bytes
-     */
-    const int kHBaseSizeOfBoolean_ = sizeof(uint8_t) / sizeof(uint8_t);
-
-    /**
-     * Size of byte in bytes
-     */
-    const uint8_t kHBaseSizeOfByte_ = kHBaseSizeOfBoolean_;
-
-    /**
-     * Size of int in bytes
-     */
-    const uint32_t kHBaseSizeOfInt_ = sizeof(uint32_t) / kHBaseSizeOfByte_;
-
-    /**
-     * Size of long in bytes
-     */
-    const uint64_t kHBaseSizeOfLong_ = sizeof(uint64_t) / kHBaseSizeOfByte_;
-
-    /**
-     * Size of Short in bytes
-     */
-    const uint16_t kHBaseSizeOfShort_ = sizeof(uint16_t) / kHBaseSizeOfByte_;
-
-    const uint32_t kHBaseSizeOfKeyLength_ = kHBaseSizeOfInt_;
-    const uint32_t kHBaseSizeOfValueLength_ = kHBaseSizeOfInt_;
-    const uint16_t kHBaseSizeOfRowLength_ = kHBaseSizeOfShort_;
-    const uint8_t kHBaseSizeOfFamilyLength_ = kHBaseSizeOfByte_;
-    const uint64_t kHBaseSizeOfTimestamp_ = kHBaseSizeOfLong_;
-    const uint8_t kHBaseSizeOfKeyType_ = kHBaseSizeOfByte_;
-    const uint32_t kHBaseSizeOfTimestampAndKey_ = kHBaseSizeOfTimestamp_ + kHBaseSizeOfKeyType_;
-    const uint32_t kHBaseSizeOfKeyInfrastructure_ =
-        kHBaseSizeOfRowLength_ + kHBaseSizeOfFamilyLength_ + kHBaseSizeOfTimestampAndKey_;
-    const uint32_t kHBaseSizeOfKeyValueInfrastructure_ =
-        kHBaseSizeOfKeyLength_ + kHBaseSizeOfValueLength_;
-
-    std::unique_ptr<folly::IOBuf> cell_block_ = nullptr;
-    uint32_t offset_ = 0;
-    uint32_t length_ = 0;
-    uint32_t cur_pos_ = 0;
-    bool end_of_cell_block_ = false;
-
-    std::shared_ptr<Cell> current_cell_ = nullptr;
-  };
-};
-
-} /* namespace hbase */


[25/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
HBASE-18725 [C++] Install header files as well as library


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/128fc306
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/128fc306
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/128fc306

Branch: refs/heads/HBASE-14850
Commit: 128fc3065117ec7c86d89dc3dacb0bc26e847639
Parents: 95da699
Author: Enis Soztutar <en...@apache.org>
Authored: Mon Sep 11 18:36:11 2017 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri Sep 15 14:19:08 2017 -0700

----------------------------------------------------------------------
 hbase-native-client/.gitignore                  |   2 +-
 hbase-native-client/Makefile                    |  54 +-
 hbase-native-client/Makefile.protos             |  14 +-
 hbase-native-client/bin/copy-protobuf.sh        |   2 +-
 hbase-native-client/bin/copy-version.sh         |   2 +-
 hbase-native-client/bin/cpplint.sh              |  11 +-
 hbase-native-client/bin/format-code.sh          |   9 +-
 hbase-native-client/connection/BUCK             |  86 --
 .../connection/client-dispatcher.cc             | 115 ---
 .../connection/client-dispatcher.h              |  77 --
 .../connection/client-handler.cc                | 145 ----
 hbase-native-client/connection/client-handler.h |  89 --
 .../connection/connection-factory.cc            |  88 --
 .../connection/connection-factory.h             |  83 --
 hbase-native-client/connection/connection-id.h  |  93 ---
 .../connection/connection-pool-test.cc          | 140 ----
 .../connection/connection-pool.cc               | 119 ---
 .../connection/connection-pool.h                |  90 --
 hbase-native-client/connection/pipeline.cc      |  55 --
 hbase-native-client/connection/pipeline.h       |  63 --
 hbase-native-client/connection/request.cc       |  46 --
 hbase-native-client/connection/request.h        |  79 --
 hbase-native-client/connection/response.h       |  93 ---
 hbase-native-client/connection/rpc-client.cc    | 119 ---
 hbase-native-client/connection/rpc-client.h     |  83 --
 hbase-native-client/connection/rpc-connection.h |  79 --
 .../connection/rpc-fault-injector-inl.h         |  41 -
 .../connection/rpc-fault-injector.cc            |  21 -
 .../connection/rpc-fault-injector.h             |  50 --
 .../connection/rpc-test-server-handler.cc       |  81 --
 .../connection/rpc-test-server-handler.h        |  47 --
 .../connection/rpc-test-server.cc               | 108 ---
 .../connection/rpc-test-server.h                |  68 --
 hbase-native-client/connection/rpc-test.cc      | 284 -------
 hbase-native-client/connection/sasl-handler.cc  | 225 -----
 hbase-native-client/connection/sasl-handler.h   |  78 --
 hbase-native-client/connection/sasl-util.cc     |  92 ---
 hbase-native-client/connection/sasl-util.h      |  46 --
 hbase-native-client/connection/service.h        |  30 -
 hbase-native-client/core/BUCK                   | 348 --------
 hbase-native-client/core/action.h               |  43 -
 hbase-native-client/core/append-test.cc         | 105 ---
 hbase-native-client/core/append.cc              |  53 --
 hbase-native-client/core/append.h               |  56 --
 .../core/async-batch-rpc-retrying-caller.cc     | 488 -----------
 .../core/async-batch-rpc-retrying-caller.h      | 185 -----
 .../core/async-batch-rpc-retrying-test.cc       | 577 -------------
 .../core/async-client-scanner.cc                | 142 ----
 hbase-native-client/core/async-client-scanner.h | 119 ---
 hbase-native-client/core/async-connection.cc    |  70 --
 hbase-native-client/core/async-connection.h     | 121 ---
 hbase-native-client/core/async-region-locator.h |  65 --
 .../core/async-rpc-retrying-caller-factory.cc   |  22 -
 .../core/async-rpc-retrying-caller-factory.h    | 343 --------
 .../core/async-rpc-retrying-caller.cc           | 230 ------
 .../core/async-rpc-retrying-caller.h            | 115 ---
 .../core/async-rpc-retrying-test.cc             | 420 ----------
 .../core/async-scan-rpc-retrying-caller.cc      | 448 ----------
 .../core/async-scan-rpc-retrying-caller.h       | 233 ------
 .../core/async-table-result-scanner.cc          | 161 ----
 .../core/async-table-result-scanner.h           |  98 ---
 hbase-native-client/core/cell-test.cc           | 195 -----
 hbase-native-client/core/cell.cc                | 123 ---
 hbase-native-client/core/cell.h                 |  72 --
 hbase-native-client/core/client-test.cc         | 697 ----------------
 hbase-native-client/core/client.cc              |  55 --
 hbase-native-client/core/client.h               |  77 --
 hbase-native-client/core/configuration-test.cc  | 119 ---
 hbase-native-client/core/configuration.cc       | 244 ------
 hbase-native-client/core/configuration.h        | 232 ------
 .../core/connection-configuration.h             | 207 -----
 hbase-native-client/core/delete-test.cc         | 124 ---
 hbase-native-client/core/delete.cc              | 131 ---
 hbase-native-client/core/delete.h               | 111 ---
 hbase-native-client/core/filter-test.cc         | 141 ----
 hbase-native-client/core/filter.h               | 479 -----------
 hbase-native-client/core/get-test.cc            | 221 -----
 hbase-native-client/core/get.cc                 | 123 ---
 hbase-native-client/core/get.h                  | 132 ---
 .../core/hbase-configuration-loader.cc          | 208 -----
 .../core/hbase-configuration-loader.h           | 140 ----
 .../core/hbase-configuration-test.cc            | 375 ---------
 .../core/hbase-rpc-controller.cc                |  22 -
 hbase-native-client/core/hbase-rpc-controller.h |  59 --
 hbase-native-client/core/increment-test.cc      | 129 ---
 hbase-native-client/core/increment.cc           |  56 --
 hbase-native-client/core/increment.h            |  57 --
 hbase-native-client/core/keyvalue-codec.cc      |  69 --
 hbase-native-client/core/keyvalue-codec.h       | 147 ----
 hbase-native-client/core/load-client.cc         | 390 ---------
 .../core/location-cache-retry-test.cc           | 112 ---
 hbase-native-client/core/location-cache-test.cc | 164 ----
 hbase-native-client/core/location-cache.cc      | 334 --------
 hbase-native-client/core/location-cache.h       | 219 -----
 hbase-native-client/core/meta-utils.cc          | 125 ---
 hbase-native-client/core/meta-utils.h           |  76 --
 hbase-native-client/core/multi-response.cc      |  86 --
 hbase-native-client/core/multi-response.h       |  79 --
 hbase-native-client/core/mutation.cc            |  69 --
 hbase-native-client/core/mutation.h             |  96 ---
 hbase-native-client/core/put-test.cc            | 135 ---
 hbase-native-client/core/put.cc                 |  68 --
 hbase-native-client/core/put.h                  |  68 --
 hbase-native-client/core/query.h                |  54 --
 hbase-native-client/core/raw-async-table.cc     | 259 ------
 hbase-native-client/core/raw-async-table.h      | 115 ---
 .../core/raw-scan-result-consumer.h             | 131 ---
 hbase-native-client/core/region-location.h      |  78 --
 hbase-native-client/core/region-request.h       |  46 --
 hbase-native-client/core/region-result.cc       |  53 --
 hbase-native-client/core/region-result.h        |  55 --
 .../core/request-converter-test.cc              | 126 ---
 hbase-native-client/core/request-converter.cc   | 368 ---------
 hbase-native-client/core/request-converter.h    | 125 ---
 hbase-native-client/core/response-converter.cc  | 221 -----
 hbase-native-client/core/response-converter.h   |  71 --
 hbase-native-client/core/result-scanner.h       |  47 --
 hbase-native-client/core/result-test.cc         | 322 --------
 hbase-native-client/core/result.cc              | 146 ----
 hbase-native-client/core/result.h               | 136 ---
 hbase-native-client/core/row.h                  |  62 --
 .../core/scan-result-cache-test.cc              | 177 ----
 hbase-native-client/core/scan-result-cache.cc   | 160 ----
 hbase-native-client/core/scan-result-cache.h    |  80 --
 hbase-native-client/core/scan-test.cc           | 228 -----
 hbase-native-client/core/scan.cc                | 186 -----
 hbase-native-client/core/scan.h                 | 270 ------
 hbase-native-client/core/scanner-test.cc        | 368 ---------
 hbase-native-client/core/server-request.h       |  62 --
 hbase-native-client/core/simple-client.cc       | 207 -----
 hbase-native-client/core/table.cc               | 144 ----
 hbase-native-client/core/table.h                | 151 ----
 hbase-native-client/core/time-range-test.cc     |  48 --
 hbase-native-client/core/time-range.cc          |  73 --
 hbase-native-client/core/time-range.h           |  58 --
 hbase-native-client/core/zk-util-test.cc        |  50 --
 hbase-native-client/core/zk-util.cc             |  62 --
 hbase-native-client/core/zk-util.h              |  46 --
 hbase-native-client/exceptions/BUCK             |  39 -
 .../exceptions/exception-test.cc                |  64 --
 hbase-native-client/exceptions/exception.cc     | 128 ---
 hbase-native-client/exceptions/exception.h      | 328 --------
 hbase-native-client/if/AccessControl.proto      | 123 ---
 hbase-native-client/if/Admin.proto              | 310 -------
 hbase-native-client/if/Aggregate.proto          |  63 --
 hbase-native-client/if/Authentication.proto     |  82 --
 hbase-native-client/if/BUCK                     |  48 --
 hbase-native-client/if/Cell.proto               |  67 --
 hbase-native-client/if/Client.proto             | 478 -----------
 hbase-native-client/if/ClusterId.proto          |  34 -
 hbase-native-client/if/ClusterStatus.proto      | 227 -----
 hbase-native-client/if/Comparator.proto         |  74 --
 hbase-native-client/if/Encryption.proto         |  33 -
 hbase-native-client/if/ErrorHandling.proto      |  58 --
 hbase-native-client/if/FS.proto                 |  45 -
 hbase-native-client/if/Filter.proto             | 171 ----
 hbase-native-client/if/HBase.proto              | 253 ------
 hbase-native-client/if/HFile.proto              |  49 --
 hbase-native-client/if/LoadBalancer.proto       |  29 -
 hbase-native-client/if/MapReduce.proto          |  37 -
 hbase-native-client/if/Master.proto             | 828 -------------------
 hbase-native-client/if/MasterProcedure.proto    | 285 -------
 hbase-native-client/if/MultiRowMutation.proto   |  45 -
 hbase-native-client/if/Procedure.proto          | 119 ---
 hbase-native-client/if/Quota.proto              |  76 --
 hbase-native-client/if/RPC.proto                | 137 ---
 hbase-native-client/if/RSGroup.proto            |  34 -
 hbase-native-client/if/RSGroupAdmin.proto       | 136 ---
 hbase-native-client/if/RegionNormalizer.proto   |  28 -
 hbase-native-client/if/RegionServerStatus.proto | 158 ----
 hbase-native-client/if/RowProcessor.proto       |  45 -
 hbase-native-client/if/SecureBulkLoad.proto     |  72 --
 hbase-native-client/if/Snapshot.proto           |  66 --
 hbase-native-client/if/Tracing.proto            |  33 -
 hbase-native-client/if/VisibilityLabels.proto   |  83 --
 hbase-native-client/if/WAL.proto                | 173 ----
 hbase-native-client/if/ZooKeeper.proto          | 176 ----
 hbase-native-client/if/test.proto               |  43 -
 hbase-native-client/if/test_rpc_service.proto   |  36 -
 hbase-native-client/include/hbase/client/BUCK   |  93 +++
 .../include/hbase/client/action.h               |  43 +
 .../include/hbase/client/append.h               |  56 ++
 .../client/async-batch-rpc-retrying-caller.h    | 185 +++++
 .../include/hbase/client/async-client-scanner.h | 119 +++
 .../include/hbase/client/async-connection.h     | 121 +++
 .../include/hbase/client/async-region-locator.h |  65 ++
 .../client/async-rpc-retrying-caller-factory.h  | 343 ++++++++
 .../hbase/client/async-rpc-retrying-caller.h    | 115 +++
 .../client/async-scan-rpc-retrying-caller.h     | 233 ++++++
 .../hbase/client/async-table-result-scanner.h   |  98 +++
 hbase-native-client/include/hbase/client/cell.h |  72 ++
 .../include/hbase/client/client.h               |  77 ++
 .../include/hbase/client/configuration.h        | 232 ++++++
 .../hbase/client/connection-configuration.h     | 207 +++++
 .../include/hbase/client/delete.h               | 111 +++
 .../include/hbase/client/filter.h               | 479 +++++++++++
 hbase-native-client/include/hbase/client/get.h  | 132 +++
 .../hbase/client/hbase-configuration-loader.h   | 140 ++++
 .../include/hbase/client/hbase-rpc-controller.h |  59 ++
 .../include/hbase/client/increment.h            |  57 ++
 .../include/hbase/client/keyvalue-codec.h       | 147 ++++
 .../include/hbase/client/location-cache.h       | 219 +++++
 .../include/hbase/client/meta-utils.h           |  76 ++
 .../include/hbase/client/multi-response.h       |  79 ++
 .../include/hbase/client/mutation.h             |  96 +++
 hbase-native-client/include/hbase/client/put.h  |  68 ++
 .../include/hbase/client/query.h                |  54 ++
 .../include/hbase/client/raw-async-table.h      | 114 +++
 .../hbase/client/raw-scan-result-consumer.h     | 131 +++
 .../include/hbase/client/region-location.h      |  78 ++
 .../include/hbase/client/region-request.h       |  46 ++
 .../include/hbase/client/region-result.h        |  55 ++
 .../include/hbase/client/request-converter.h    | 125 +++
 .../include/hbase/client/response-converter.h   |  71 ++
 .../include/hbase/client/result-scanner.h       |  47 ++
 .../include/hbase/client/result.h               | 136 +++
 hbase-native-client/include/hbase/client/row.h  |  62 ++
 .../include/hbase/client/scan-result-cache.h    |  80 ++
 hbase-native-client/include/hbase/client/scan.h | 270 ++++++
 .../include/hbase/client/server-request.h       |  62 ++
 .../include/hbase/client/table.h                | 151 ++++
 .../include/hbase/client/time-range.h           |  58 ++
 .../include/hbase/client/zk-util.h              |  46 ++
 .../include/hbase/connection/BUCK               |  57 ++
 .../hbase/connection/client-dispatcher.h        |  77 ++
 .../include/hbase/connection/client-handler.h   |  89 ++
 .../hbase/connection/connection-factory.h       |  83 ++
 .../include/hbase/connection/connection-id.h    |  93 +++
 .../include/hbase/connection/connection-pool.h  |  90 ++
 .../include/hbase/connection/pipeline.h         |  63 ++
 .../include/hbase/connection/request.h          |  79 ++
 .../include/hbase/connection/response.h         |  93 +++
 .../include/hbase/connection/rpc-client.h       |  83 ++
 .../include/hbase/connection/rpc-connection.h   |  79 ++
 .../hbase/connection/rpc-fault-injector-inl.h   |  41 +
 .../hbase/connection/rpc-fault-injector.h       |  50 ++
 .../hbase/connection/rpc-test-server-handler.h  |  47 ++
 .../include/hbase/connection/rpc-test-server.h  |  68 ++
 .../include/hbase/connection/sasl-handler.h     |  78 ++
 .../include/hbase/connection/sasl-util.h        |  46 ++
 .../include/hbase/connection/service.h          |  30 +
 .../include/hbase/exceptions/BUCK               |  28 +
 .../include/hbase/exceptions/exception.h        | 328 ++++++++
 hbase-native-client/include/hbase/security/BUCK |  30 +
 .../include/hbase/security/user.h               |  46 ++
 hbase-native-client/include/hbase/serde/BUCK    |  37 +
 .../include/hbase/serde/cell-outputstream.h     |  53 ++
 .../include/hbase/serde/cell-scanner.h          |  50 ++
 hbase-native-client/include/hbase/serde/codec.h |  50 ++
 .../include/hbase/serde/region-info.h           |  44 +
 .../include/hbase/serde/rpc-serde.h             | 155 ++++
 .../include/hbase/serde/server-name.h           |  47 ++
 .../include/hbase/serde/table-name.h            |  58 ++
 hbase-native-client/include/hbase/serde/zk.h    |  49 ++
 hbase-native-client/include/hbase/utils/BUCK    |  40 +
 .../include/hbase/utils/bytes-util.h            |  68 ++
 .../include/hbase/utils/concurrent-map.h        | 130 +++
 .../include/hbase/utils/connection-util.h       |  62 ++
 .../include/hbase/utils/optional.h              |  32 +
 .../include/hbase/utils/sys-util.h              |  39 +
 .../include/hbase/utils/time-util.h             |  71 ++
 .../include/hbase/utils/user-util.h             |  54 ++
 hbase-native-client/security/BUCK               |  30 -
 hbase-native-client/security/user.h             |  46 --
 hbase-native-client/serde/BUCK                  |  96 ---
 hbase-native-client/serde/cell-outputstream.h   |  53 --
 hbase-native-client/serde/cell-scanner.h        |  50 --
 .../serde/client-deserializer-test.cc           |  64 --
 .../serde/client-serializer-test.cc             |  75 --
 hbase-native-client/serde/codec.h               |  50 --
 .../serde/region-info-deserializer-test.cc      |  53 --
 hbase-native-client/serde/region-info.h         |  44 -
 hbase-native-client/serde/rpc-serde.cc          | 261 ------
 hbase-native-client/serde/rpc-serde.h           | 155 ----
 hbase-native-client/serde/server-name-test.cc   |  47 --
 hbase-native-client/serde/server-name.h         |  47 --
 hbase-native-client/serde/table-name-test.cc    |  54 --
 hbase-native-client/serde/table-name.h          |  58 --
 .../serde/zk-deserializer-test.cc               | 123 ---
 hbase-native-client/serde/zk.cc                 |  77 --
 hbase-native-client/serde/zk.h                  |  49 --
 hbase-native-client/src/hbase/client/BUCK       | 301 +++++++
 .../src/hbase/client/append-test.cc             | 105 +++
 hbase-native-client/src/hbase/client/append.cc  |  53 ++
 .../client/async-batch-rpc-retrying-caller.cc   | 488 +++++++++++
 .../client/async-batch-rpc-retrying-test.cc     | 577 +++++++++++++
 .../src/hbase/client/async-client-scanner.cc    | 142 ++++
 .../src/hbase/client/async-connection.cc        |  70 ++
 .../client/async-rpc-retrying-caller-factory.cc |  22 +
 .../hbase/client/async-rpc-retrying-caller.cc   | 230 ++++++
 .../src/hbase/client/async-rpc-retrying-test.cc | 420 ++++++++++
 .../client/async-scan-rpc-retrying-caller.cc    | 448 ++++++++++
 .../hbase/client/async-table-result-scanner.cc  | 161 ++++
 .../src/hbase/client/cell-test.cc               | 195 +++++
 hbase-native-client/src/hbase/client/cell.cc    | 123 +++
 .../src/hbase/client/client-test.cc             | 697 ++++++++++++++++
 hbase-native-client/src/hbase/client/client.cc  |  55 ++
 .../src/hbase/client/configuration-test.cc      | 119 +++
 .../src/hbase/client/configuration.cc           | 244 ++++++
 .../src/hbase/client/delete-test.cc             | 124 +++
 hbase-native-client/src/hbase/client/delete.cc  | 131 +++
 .../src/hbase/client/filter-test.cc             | 141 ++++
 .../src/hbase/client/get-test.cc                | 221 +++++
 hbase-native-client/src/hbase/client/get.cc     | 123 +++
 .../hbase/client/hbase-configuration-loader.cc  | 208 +++++
 .../hbase/client/hbase-configuration-test.cc    | 375 +++++++++
 .../src/hbase/client/hbase-rpc-controller.cc    |  22 +
 .../src/hbase/client/increment-test.cc          | 129 +++
 .../src/hbase/client/increment.cc               |  56 ++
 .../src/hbase/client/keyvalue-codec.cc          |  69 ++
 .../src/hbase/client/load-client.cc             | 390 +++++++++
 .../hbase/client/location-cache-retry-test.cc   | 112 +++
 .../src/hbase/client/location-cache-test.cc     | 164 ++++
 .../src/hbase/client/location-cache.cc          | 334 ++++++++
 .../src/hbase/client/meta-utils.cc              | 125 +++
 .../src/hbase/client/multi-response.cc          |  86 ++
 .../src/hbase/client/mutation.cc                |  69 ++
 .../src/hbase/client/put-test.cc                | 135 +++
 hbase-native-client/src/hbase/client/put.cc     |  68 ++
 .../src/hbase/client/raw-async-table.cc         | 260 ++++++
 .../src/hbase/client/region-result.cc           |  53 ++
 .../src/hbase/client/request-converter-test.cc  | 126 +++
 .../src/hbase/client/request-converter.cc       | 368 +++++++++
 .../src/hbase/client/response-converter.cc      | 221 +++++
 .../src/hbase/client/result-test.cc             | 322 ++++++++
 hbase-native-client/src/hbase/client/result.cc  | 146 ++++
 .../src/hbase/client/scan-result-cache-test.cc  | 177 ++++
 .../src/hbase/client/scan-result-cache.cc       | 160 ++++
 .../src/hbase/client/scan-test.cc               | 228 +++++
 hbase-native-client/src/hbase/client/scan.cc    | 186 +++++
 .../src/hbase/client/scanner-test.cc            | 368 +++++++++
 .../src/hbase/client/simple-client.cc           | 207 +++++
 hbase-native-client/src/hbase/client/table.cc   | 144 ++++
 .../src/hbase/client/time-range-test.cc         |  48 ++
 .../src/hbase/client/time-range.cc              |  73 ++
 .../src/hbase/client/zk-util-test.cc            |  50 ++
 hbase-native-client/src/hbase/client/zk-util.cc |  62 ++
 hbase-native-client/src/hbase/connection/BUCK   |  68 ++
 .../src/hbase/connection/client-dispatcher.cc   | 115 +++
 .../src/hbase/connection/client-handler.cc      | 145 ++++
 .../src/hbase/connection/connection-factory.cc  |  88 ++
 .../hbase/connection/connection-pool-test.cc    | 140 ++++
 .../src/hbase/connection/connection-pool.cc     | 119 +++
 .../src/hbase/connection/pipeline.cc            |  55 ++
 .../src/hbase/connection/request.cc             |  46 ++
 .../src/hbase/connection/rpc-client.cc          | 119 +++
 .../src/hbase/connection/rpc-fault-injector.cc  |  21 +
 .../hbase/connection/rpc-test-server-handler.cc |  81 ++
 .../src/hbase/connection/rpc-test-server.cc     | 108 +++
 .../src/hbase/connection/rpc-test.cc            | 284 +++++++
 .../src/hbase/connection/sasl-handler.cc        | 225 +++++
 .../src/hbase/connection/sasl-util.cc           |  92 +++
 hbase-native-client/src/hbase/exceptions/BUCK   |  37 +
 .../src/hbase/exceptions/exception-test.cc      |  64 ++
 .../src/hbase/exceptions/exception.cc           | 128 +++
 .../src/hbase/if/AccessControl.proto            | 123 +++
 hbase-native-client/src/hbase/if/Admin.proto    | 310 +++++++
 .../src/hbase/if/Aggregate.proto                |  63 ++
 .../src/hbase/if/Authentication.proto           |  82 ++
 hbase-native-client/src/hbase/if/BUCK           |  49 ++
 hbase-native-client/src/hbase/if/Cell.proto     |  67 ++
 hbase-native-client/src/hbase/if/Client.proto   | 478 +++++++++++
 .../src/hbase/if/ClusterId.proto                |  34 +
 .../src/hbase/if/ClusterStatus.proto            | 227 +++++
 .../src/hbase/if/Comparator.proto               |  74 ++
 .../src/hbase/if/Encryption.proto               |  33 +
 .../src/hbase/if/ErrorHandling.proto            |  58 ++
 hbase-native-client/src/hbase/if/FS.proto       |  45 +
 hbase-native-client/src/hbase/if/Filter.proto   | 171 ++++
 hbase-native-client/src/hbase/if/HBase.proto    | 253 ++++++
 hbase-native-client/src/hbase/if/HFile.proto    |  49 ++
 .../src/hbase/if/LoadBalancer.proto             |  29 +
 .../src/hbase/if/MapReduce.proto                |  37 +
 hbase-native-client/src/hbase/if/Master.proto   | 828 +++++++++++++++++++
 .../src/hbase/if/MasterProcedure.proto          | 285 +++++++
 .../src/hbase/if/MultiRowMutation.proto         |  45 +
 .../src/hbase/if/Procedure.proto                | 119 +++
 hbase-native-client/src/hbase/if/Quota.proto    |  76 ++
 hbase-native-client/src/hbase/if/RPC.proto      | 137 +++
 hbase-native-client/src/hbase/if/RSGroup.proto  |  34 +
 .../src/hbase/if/RSGroupAdmin.proto             | 136 +++
 .../src/hbase/if/RegionNormalizer.proto         |  28 +
 .../src/hbase/if/RegionServerStatus.proto       | 158 ++++
 .../src/hbase/if/RowProcessor.proto             |  45 +
 .../src/hbase/if/SecureBulkLoad.proto           |  72 ++
 hbase-native-client/src/hbase/if/Snapshot.proto |  66 ++
 hbase-native-client/src/hbase/if/Tracing.proto  |  33 +
 .../src/hbase/if/VisibilityLabels.proto         |  83 ++
 hbase-native-client/src/hbase/if/WAL.proto      | 173 ++++
 .../src/hbase/if/ZooKeeper.proto                | 176 ++++
 hbase-native-client/src/hbase/if/test.proto     |  43 +
 .../src/hbase/if/test_rpc_service.proto         |  36 +
 hbase-native-client/src/hbase/security/BUCK     |  27 +
 hbase-native-client/src/hbase/serde/BUCK        |  86 ++
 .../src/hbase/serde/client-deserializer-test.cc |  64 ++
 .../src/hbase/serde/client-serializer-test.cc   |  75 ++
 .../serde/region-info-deserializer-test.cc      |  53 ++
 .../src/hbase/serde/rpc-serde.cc                | 261 ++++++
 .../src/hbase/serde/server-name-test.cc         |  47 ++
 .../src/hbase/serde/table-name-test.cc          |  54 ++
 .../src/hbase/serde/zk-deserializer-test.cc     | 123 +++
 hbase-native-client/src/hbase/serde/zk.cc       |  77 ++
 hbase-native-client/src/hbase/test-util/BUCK    |  53 ++
 .../src/hbase/test-util/mini-cluster.cc         | 311 +++++++
 .../src/hbase/test-util/mini-cluster.h          |  81 ++
 .../src/hbase/test-util/test-util.cc            | 105 +++
 .../src/hbase/test-util/test-util.h             |  78 ++
 hbase-native-client/src/hbase/utils/BUCK        |  57 ++
 .../src/hbase/utils/bytes-util-test.cc          |  69 ++
 .../src/hbase/utils/bytes-util.cc               |  88 ++
 .../src/hbase/utils/concurrent-map-test.cc      |  36 +
 .../src/hbase/utils/connection-util.cc          |  26 +
 .../src/hbase/utils/user-util-test.cc           |  35 +
 .../src/hbase/utils/user-util.cc                |  77 ++
 hbase-native-client/test-util/BUCK              |  52 --
 hbase-native-client/test-util/mini-cluster.cc   | 311 -------
 hbase-native-client/test-util/mini-cluster.h    |  81 --
 hbase-native-client/test-util/test-util.cc      | 105 ---
 hbase-native-client/test-util/test-util.h       |  78 --
 hbase-native-client/utils/BUCK                  |  64 --
 hbase-native-client/utils/bytes-util-test.cc    |  69 --
 hbase-native-client/utils/bytes-util.cc         |  88 --
 hbase-native-client/utils/bytes-util.h          |  68 --
 .../utils/concurrent-map-test.cc                |  36 -
 hbase-native-client/utils/concurrent-map.h      | 130 ---
 hbase-native-client/utils/connection-util.cc    |  26 -
 hbase-native-client/utils/connection-util.h     |  62 --
 hbase-native-client/utils/optional.h            |  32 -
 hbase-native-client/utils/sys-util.h            |  39 -
 hbase-native-client/utils/time-util.h           |  71 --
 hbase-native-client/utils/user-util-test.cc     |  35 -
 hbase-native-client/utils/user-util.cc          |  77 --
 hbase-native-client/utils/user-util.h           |  54 --
 433 files changed, 27390 insertions(+), 27162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/.gitignore
----------------------------------------------------------------------
diff --git a/hbase-native-client/.gitignore b/hbase-native-client/.gitignore
index c99ce59..e8dd42d 100644
--- a/hbase-native-client/.gitignore
+++ b/hbase-native-client/.gitignore
@@ -26,4 +26,4 @@ third-party/*
 /gcc-debug/
 
 # Generated files
-utils/version.h
+src/hbase/utils/version.h

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/Makefile
----------------------------------------------------------------------
diff --git a/hbase-native-client/Makefile b/hbase-native-client/Makefile
index a291dfa..4d6430c 100644
--- a/hbase-native-client/Makefile
+++ b/hbase-native-client/Makefile
@@ -19,17 +19,20 @@
 CC := g++
 LD := g++
 
+SRC_HBASE := src/hbase
+INCLUDE_HBASE := include/
 BUILD_PATH := build
 DEBUG_PATH := $(BUILD_PATH)/debug
 RELEASE_PATH := $(BUILD_PATH)/release
-PROTO_SRC_DIR := if
-PROTO_CXX_DIR := $(BUILD_PATH)/if
-MODULES := connection core exceptions security serde utils
+PROTO_SRC_DIR := $(SRC_HBASE)/if
+PROTO_CXX_DIR := $(BUILD_PATH)/$(PROTO_SRC_DIR)
+MODULES := connection client exceptions security serde utils
 TEST_MODULES := test-util # These modules contain test code, not included in the build for the lib
-SRC_DIR := $(MODULES)
-DEBUG_BUILD_DIR := $(addprefix $(DEBUG_PATH)/,$(MODULES))
-RELEASE_BUILD_DIR := $(addprefix $(RELEASE_PATH)/,$(MODULES))
-INCLUDE_DIR := . $(BUILD_PATH)
+SRC_DIR := $(addprefix $(SRC_HBASE)/,$(MODULES))
+DEBUG_BUILD_DIR := $(addprefix $(DEBUG_PATH)/hbase/,$(MODULES))
+RELEASE_BUILD_DIR := $(addprefix $(RELEASE_PATH)/hbase/,$(MODULES))
+
+INCLUDE_DIR := . src $(BUILD_PATH)/src $(INCLUDE_HBASE)
 TEST_BUILD_INCLUDE_DIR := $(INLCUDE_DIR) $(JAVA_HOME)/include/ $(JAVA_HOME)/include/linux
 
 #flags to pass to the CPP compiler & linker
@@ -42,14 +45,14 @@ LINKFLAG := -shared
 #define list of source files and object files
 ALLSRC := $(foreach sdir,$(SRC_DIR),$(wildcard $(sdir)/*.cc))
 EXCLUDE_SRC := $(foreach sdir,$(SRC_DIR),$(wildcard $(sdir)/*-test.cc)) \
-	core/simple-client.cc core/load-client.cc
+	$(SRC_HBASE)/client/simple-client.cc $(SRC_HBASE)/client/load-client.cc
 SRC := $(filter-out $(EXCLUDE_SRC), $(ALLSRC))
-PROTOSRC := $(patsubst %.proto, $(addprefix $(BUILD_PATH)/,%.pb.cc),$(wildcard if/*.proto))
-PROTOHDR := $(patsubst %.proto, $(addprefix $(BUILD_PATH)/,%.pb.h),$(wildcard if/*.proto))
-DEBUG_OBJ := $(patsubst %.cc,$(DEBUG_PATH)/%.o,$(SRC))
-DEBUG_OBJ += $(patsubst %.cc,$(DEBUG_PATH)/%.o,$(PROTOSRC))
-RELEASE_OBJ := $(patsubst %.cc,$(RELEASE_PATH)/%.o,$(SRC))
-RELEASE_OBJ += $(patsubst %.cc,$(RELEASE_PATH)/%.o,$(PROTOSRC))
+PROTOSRC := $(patsubst $(PROTO_SRC_DIR)/%.proto, $(addprefix $(PROTO_CXX_DIR)/,%.pb.cc),$(wildcard $(PROTO_SRC_DIR)/*.proto))
+PROTOHDR := $(patsubst $(PROTO_SRC_DIR)/%.proto, $(addprefix $(PROTO_CXX_DIR)/,%.pb.h),$(wildcard $(PROTO_SRC_DIR)/*.proto))
+DEBUG_OBJ := $(patsubst $(SRC_HBASE)/%.cc,$(DEBUG_PATH)/hbase/%.o,$(SRC))
+DEBUG_OBJ += $(patsubst $(PROTO_CXX_DIR)/%.cc,$(DEBUG_PATH)/hbase/if/%.o,$(PROTOSRC))
+RELEASE_OBJ := $(patsubst $(SRC_HBASE)/%.cc,$(RELEASE_PATH)/hbase/%.o,$(SRC))
+RELEASE_OBJ += $(patsubst $(PROTO_CXX_DIR)/%.cc,$(RELEASE_PATH)/hbase/if/%.o,$(PROTOSRC))
 INCLUDES := $(addprefix -I,$(INCLUDE_DIR))
 
 LIB_DIR := /usr/local
@@ -59,8 +62,9 @@ LIB_RELEASE := $(RELEASE_PATH)/libHBaseClient.so
 ARC_RELEASE := $(RELEASE_PATH)/libHBaseClient.a
 LIB_DEBUG := $(DEBUG_PATH)/libHBaseClient_d.so
 ARC_DEBUG := $(DEBUG_PATH)/libHBaseClient_d.a
+LOCAL_INCLUDE_DIR := /usr/local/include/
 
-build: checkdirs protos $(LIB_DEBUG) $(LIB_RELEASE) $(ARC_DEBUG) $(ARC_RELEASE)
+build: checkdirs protos copyfiles $(LIB_DEBUG) $(LIB_RELEASE) $(ARC_DEBUG) $(ARC_RELEASE)
 
 vpath %.cc $(SRC_DIR)
 
@@ -80,7 +84,7 @@ $1/%.o: %.cc
 	$(CC) -c $$< -o $$@ -MF$$(@:%.o=%.d) -MT$$@ $(CPPFLAGS_RELEASE) $(INCLUDES)
 endef
 
-.PHONY: all clean install 
+.PHONY: all clean install copyfiles
 
 checkdirs: $(DEBUG_BUILD_DIR) $(RELEASE_BUILD_DIR) $(PROTO_CXX_DIR)
 
@@ -88,22 +92,32 @@ copyfiles:
 	@bin/copy-protobuf.sh
 	@bin/copy-version.sh
 
+# .proto files are in src/hbase/if. These are compiled into C++ code by the 
+# protoc compiler, and turned into .cc and .h files under build/src/hbase/if
 $(PROTO_CXX_DIR)/%.pb.cc $(PROTO_CXX_DIR)/%.pb.h: $(PROTO_SRC_DIR)/%.proto
 	@protoc --proto_path=$(PROTO_SRC_DIR) --cpp_out=$(PROTO_CXX_DIR) $<
 
-#Run parallel jobs to speed up compilation
+# protos target compiles the .cc and .h files into .o files for the protobuf
+# generated source files
 protos: $(PROTO_CXX_DIR) $(PROTOSRC) $(PROTOHDR)
 	@make -j8 all -f Makefile.protos
 
-install:
+install_headers:
+	cp -r $(INCLUDE_HBASE)/hbase $(LOCAL_INCLUDE_DIR)
+	cp -r $(PROTO_CXX_DIR) $(LOCAL_INCLUDE_DIR)/hbase/
+
+uninstall_headers:
+	rm -rf $(LOCAL_INCLUDE_DIR)/hbase
+
+install: install_headers
 	cp $(LIB_RELEASE) $(LIB_LIBDIR)/libHBaseClient.so
 	cp $(ARC_RELEASE) $(LIB_LIBDIR)/libHBaseClient.a
 	cp $(LIB_DEBUG) $(LIB_LIBDIR)/libHBaseClient_d.so
 	cp $(ARC_DEBUG) $(LIB_LIBDIR)/libHBaseClient_d.a
 	ldconfig
 
-uninstall:
-	rm -f $(LIB_LIBDIR)/libHBaseClient.so $(LIB_LIBDIR)/libHBaseClient.a $(LIB_LIBDIR)/libHBaseClient_d.so $(ARC_DEBUG) $(LIB_LIBDIR)/libHBaseClient_d.a
+uninstall: uninstall_headers
+	rm -f $(LIB_LIBDIR)/libHBaseClient.so $(LIB_LIBDIR)/libHBaseClient.a $(LIB_LIBDIR)/libHBaseClient_d.so $(LIB_LIBDIR)/libHBaseClient_d.a
 	ldconfig
 
 $(PROTO_CXX_DIR):

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/Makefile.protos
----------------------------------------------------------------------
diff --git a/hbase-native-client/Makefile.protos b/hbase-native-client/Makefile.protos
index 4cf8982..8712029 100644
--- a/hbase-native-client/Makefile.protos
+++ b/hbase-native-client/Makefile.protos
@@ -21,11 +21,11 @@ CC := g++
 BUILD_PATH := build
 DEBUG_PATH := $(BUILD_PATH)/debug
 RELEASE_PATH := $(BUILD_PATH)/release
-MODULES := $(BUILD_PATH)/if
-SRC_DIR := $(MODULES)
-DEBUG_BUILD_DIR := $(addprefix $(DEBUG_PATH)/,$(MODULES))
-RELEASE_BUILD_DIR := $(addprefix $(RELEASE_PATH)/,$(MODULES))
-INCLUDE_DIR := . $(BUILD_PATH)/if
+MODULES := if
+SRC_DIR := $(BUILD_PATH)/src/hbase/$(MODULES)
+DEBUG_BUILD_DIR := $(addprefix $(DEBUG_PATH)/hbase/,$(MODULES))
+RELEASE_BUILD_DIR := $(addprefix $(RELEASE_PATH)/hbase/,$(MODULES))
+INCLUDE_DIR := . $(BUILD_PATH)/src/hbase/if
 
 #flags to pass to the CPP compiler & linker
 CPPFLAGS_DEBUG := -D_GLIBCXX_USE_CXX11_ABI=0 -g -Wall -std=c++14 -pedantic -fPIC -MMD -MP
@@ -33,8 +33,8 @@ CPPFLAGS_RELEASE := -D_GLIBCXX_USE_CXX11_ABI=0 -DNDEBUG -O2 -Wall -std=c++14 -pe
 
 #define list of source files and object files
 SRC := $(foreach sdir,$(SRC_DIR),$(wildcard $(sdir)/*.cc))
-DEBUG_OBJ := $(patsubst %.cc,$(DEBUG_PATH)/%.o,$(SRC))
-RELEASE_OBJ := $(patsubst %.cc,$(RELEASE_PATH)/%.o,$(SRC))
+DEBUG_OBJ := $(patsubst $(SRC_DIR)/%.cc,$(DEBUG_PATH)/hbase/if/%.o,$(SRC))
+RELEASE_OBJ := $(patsubst $(SRC_DIR)/%.cc,$(RELEASE_PATH)/hbase/if/%.o,$(SRC))
 INCLUDES := $(addprefix -I,$(INCLUDE_DIR))
 
 all: $(DEBUG_BUILD_DIR) $(RELEASE_BUILD_DIR) $(DEBUG_OBJ) $(RELEASE_OBJ)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/bin/copy-protobuf.sh
----------------------------------------------------------------------
diff --git a/hbase-native-client/bin/copy-protobuf.sh b/hbase-native-client/bin/copy-protobuf.sh
index f727800..47544c8 100755
--- a/hbase-native-client/bin/copy-protobuf.sh
+++ b/hbase-native-client/bin/copy-protobuf.sh
@@ -20,5 +20,5 @@ IFS=$'\n\t'
 
 BIN_DIR=$(dirname "$0")
 PB_SOURCE_DIR="${BIN_DIR}/../../hbase-protocol/src/main/protobuf/"
-PB_DEST_DIR="${BIN_DIR}/../if/"
+PB_DEST_DIR="${BIN_DIR}/../src/hbase/if/"
 rsync -r --exclude BUCK ${PB_SOURCE_DIR} ${PB_DEST_DIR}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/bin/copy-version.sh
----------------------------------------------------------------------
diff --git a/hbase-native-client/bin/copy-version.sh b/hbase-native-client/bin/copy-version.sh
index b33db7a..e2b707f 100755
--- a/hbase-native-client/bin/copy-version.sh
+++ b/hbase-native-client/bin/copy-version.sh
@@ -21,5 +21,5 @@ IFS=$'\n\t'
 # Copy the version.h generated from hbase-common/src/saveVersion.sh script via the mvn build
 BIN_DIR=$(dirname "$0")
 VERSION_SOURCE_DIR="${BIN_DIR}/../../hbase-common/target/generated-sources/native/utils/"
-VERSION_DEST_DIR="${BIN_DIR}/../utils/"
+VERSION_DEST_DIR="${BIN_DIR}/../include/hbase/utils/"
 cp $VERSION_SOURCE_DIR/* $VERSION_DEST_DIR/

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/bin/cpplint.sh
----------------------------------------------------------------------
diff --git a/hbase-native-client/bin/cpplint.sh b/hbase-native-client/bin/cpplint.sh
index 3684c2e..7a27945 100755
--- a/hbase-native-client/bin/cpplint.sh
+++ b/hbase-native-client/bin/cpplint.sh
@@ -20,6 +20,8 @@ IFS=$'\n\t'
 CPPLINT_LOC=https://raw.githubusercontent.com/google/styleguide/gh-pages/cpplint/cpplint.py
 OUTPUT=build/cpplint.py
 
+declare -a MODULES=( client connection exceptions security serde utils test-util )
+
 # Download if not already there
 wget -nc $CPPLINT_LOC -O $OUTPUT
 
@@ -27,4 +29,11 @@ wget -nc $CPPLINT_LOC -O $OUTPUT
 # Exclude the following rules: build/header_guard (We use #pragma once instead)
 #                              readability/todo (TODOs are generic)
 #                              build/c++11 (We are building with c++14)
-find core connection exceptions serde utils test-util security -name "*.h" -or -name "*.cc" | xargs -P8 python $OUTPUT --filter=-build/header_guard,-readability/todo,-build/c++11 --linelength=100
+for m in ${MODULES[@]}; do
+  if [ $m != "security" ]; then  #These are empty
+    exec find src/hbase/$m -name "*.h" -or -name "*.cc" | xargs -P8 python $OUTPUT --filter=-build/header_guard,-readability/todo,-build/c++11 --linelength=100
+  fi
+  if [ $m != "test-util" ]; then
+    exec find include/hbase/$m -name "*.h" -or -name "*.cc" | xargs -P8 python $OUTPUT --filter=-build/header_guard,-readability/todo,-build/c++11 --linelength=100
+  fi
+done

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/bin/format-code.sh
----------------------------------------------------------------------
diff --git a/hbase-native-client/bin/format-code.sh b/hbase-native-client/bin/format-code.sh
index fe236d8..301167e 100755
--- a/hbase-native-client/bin/format-code.sh
+++ b/hbase-native-client/bin/format-code.sh
@@ -18,6 +18,11 @@
 set -euo pipefail
 IFS=$'\n\t'
 
+declare -a MODULES=( client connection exceptions security serde utils test-util )
 
-find core connection exceptions serde utils test-util security -name "*.h" -or -name "*.cc" | xargs -P8 clang-format -i --style='{BasedOnStyle: Google, ColumnLimit: 100}'
-find core connection exceptions serde utils test-util third-party security -name "BUCK" | xargs -P8 yapf -i --style=google
+for m in ${MODULES[@]}; do
+  find src/hbase/$m -name "*.h" -or -name "*.cc" | xargs -P8 clang-format -i --style='{BasedOnStyle: Google, ColumnLimit: 100}'
+  find src/hbase/$m -name "BUCK" | xargs -P8 yapf -i --style=google
+done
+
+find third-party -name "BUCK" | xargs -P8 yapf -i --style=google

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/BUCK b/hbase-native-client/connection/BUCK
deleted file mode 100644
index a87d27a..0000000
--- a/hbase-native-client/connection/BUCK
+++ /dev/null
@@ -1,86 +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.
-
-# This is the library dealing with a single connection
-# to a single server.
-cxx_library(
-    name="connection",
-    exported_headers=[
-        "client-dispatcher.h",
-        "client-handler.h",
-        "sasl-handler.h",
-        "connection-factory.h",
-        "connection-pool.h",
-        "connection-id.h",
-        "pipeline.h",
-        "request.h",
-        "rpc-connection.h",
-        "response.h",
-        "service.h",
-        "rpc-client.h",
-        "sasl-util.h",
-        "rpc-test-server.h",
-        "rpc-test-server-handler.h",
-        "rpc-fault-injector.h",
-        "rpc-fault-injector-inl.h",
-    ],
-    srcs=[
-        "client-dispatcher.cc",
-        "client-handler.cc",
-        "connection-factory.cc",
-        "connection-pool.cc",
-        "pipeline.cc",
-        "request.cc",
-        "rpc-client.cc",
-        "sasl-handler.cc",
-        "sasl-util.cc",
-        "rpc-test-server.cc",
-        "rpc-test-server-handler.cc",
-        "rpc-fault-injector.cc",
-    ],
-    deps=[
-        "//if:if",
-        "//utils:utils",
-        "//serde:serde",
-        "//security:security",
-        "//third-party:folly",
-        "//third-party:wangle",
-        "//exceptions:exceptions",
-    ],
-    compiler_flags=['-Weffc++'],
-    linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
-    exported_linker_flags=['-L/usr/local/lib', '-lsasl2', '-lkrb5'],
-    visibility=[
-        '//core/...',
-    ],)
-cxx_test(
-    name="connection-pool-test",
-    srcs=[
-        "connection-pool-test.cc",
-    ],
-    deps=[
-        ":connection",
-    ],)
-cxx_test(
-    name="rpc-test",
-    srcs=[
-        "rpc-test.cc",
-    ],
-    deps=[
-        ":connection",
-    ],
-    run_test_separately=True,)

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/client-dispatcher.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/client-dispatcher.cc b/hbase-native-client/connection/client-dispatcher.cc
deleted file mode 100644
index fc8eb16..0000000
--- a/hbase-native-client/connection/client-dispatcher.cc
+++ /dev/null
@@ -1,115 +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 "connection/client-dispatcher.h"
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/Format.h>
-#include <folly/io/async/AsyncSocketException.h>
-#include <utility>
-
-#include "connection/rpc-connection.h"
-#include "exceptions/exception.h"
-
-using std::unique_ptr;
-
-namespace hbase {
-
-ClientDispatcher::ClientDispatcher(const std::string &server)
-    : current_call_id_(9), requests_(5000), server_(server), is_closed_(false) {}
-
-void ClientDispatcher::read(Context *ctx, unique_ptr<Response> in) {
-  VLOG(5) << "ClientDispatcher::read()";
-  auto call_id = in->call_id();
-  auto p = requests_.find_and_erase(call_id);
-
-  VLOG(3) << folly::sformat("Read hbase::Response, call_id: {}, hasException: {}, what: {}",
-                            in->call_id(), bool(in->exception()), in->exception().what());
-
-  if (in->exception()) {
-    p.setException(in->exception());
-  } else {
-    p.setValue(std::move(in));
-  }
-}
-
-void ClientDispatcher::readException(Context *ctx, folly::exception_wrapper e) {
-  VLOG(5) << "ClientDispatcher::readException()";
-  CloseAndCleanUpCalls();
-}
-
-void ClientDispatcher::readEOF(Context *ctx) {
-  VLOG(5) << "ClientDispatcher::readEOF()";
-  CloseAndCleanUpCalls();
-}
-
-folly::Future<unique_ptr<Response>> ClientDispatcher::operator()(unique_ptr<Request> arg) {
-  VLOG(5) << "ClientDispatcher::operator()";
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
-  if (is_closed_) {
-    throw ConnectionException("Connection closed already");
-  }
-
-  auto call_id = current_call_id_++;
-  arg->set_call_id(call_id);
-
-  // TODO: if the map is full (or we have more than hbase.client.perserver.requests.threshold)
-  // then throw ServerTooBusyException so that upper layers will retry.
-  auto &p = requests_[call_id];
-
-  auto f = p.getFuture();
-  p.setInterruptHandler([call_id, this](const folly::exception_wrapper &e) {
-    LOG(ERROR) << "e = " << call_id;
-    this->requests_.erase(call_id);
-    // TODO: call Promise::SetException()?
-  });
-
-  try {
-    this->pipeline_->write(std::move(arg));
-  } catch (const folly::AsyncSocketException &e) {
-    p.setException(folly::exception_wrapper{ConnectionException{folly::exception_wrapper{e}}});
-    /* clear folly::Promise to avoid overflow. */
-    requests_.erase(call_id);
-  }
-
-  return f;
-}
-
-void ClientDispatcher::CloseAndCleanUpCalls() {
-  VLOG(5) << "ClientDispatcher::CloseAndCleanUpCalls()";
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
-  if (is_closed_) {
-    return;
-  }
-  for (auto &pair : requests_) {
-    pair.second.setException(IOException{"Connection closed to server:" + server_});
-  }
-  requests_.clear();
-  is_closed_ = true;
-}
-
-folly::Future<folly::Unit> ClientDispatcher::close() {
-  CloseAndCleanUpCalls();
-  return ClientDispatcherBase::close();
-}
-
-folly::Future<folly::Unit> ClientDispatcher::close(Context *ctx) {
-  CloseAndCleanUpCalls();
-  return ClientDispatcherBase::close(ctx);
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/client-dispatcher.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/client-dispatcher.h b/hbase-native-client/connection/client-dispatcher.h
deleted file mode 100644
index 7ef3759..0000000
--- a/hbase-native-client/connection/client-dispatcher.h
+++ /dev/null
@@ -1,77 +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.
- *
- */
-
-#pragma once
-
-#include <folly/Logging.h>
-#include <wangle/service/ClientDispatcher.h>
-
-#include <atomic>
-#include <map>
-#include <memory>
-#include <mutex>
-#include <string>
-
-#include "connection/pipeline.h"
-#include "connection/request.h"
-#include "connection/response.h"
-#include "utils/concurrent-map.h"
-
-namespace hbase {
-
-/**
- * Dispatcher that assigns a call_id and then routes the response back to the
- * future.
- */
-class ClientDispatcher
-    : public wangle::ClientDispatcherBase<SerializePipeline, std::unique_ptr<Request>,
-                                          std::unique_ptr<Response>> {
- public:
-  /** Create a new ClientDispatcher */
-  explicit ClientDispatcher(const std::string &server);
-  /** Read a response off the pipeline. */
-  void read(Context *ctx, std::unique_ptr<Response> in) override;
-  void readException(Context *ctx, folly::exception_wrapper e) override;
-  void readEOF(Context *ctx) override;
-  /** Take a request as a call and send it down the pipeline. */
-  folly::Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> arg) override;
-  /** Close the dispatcher and the associated pipeline. */
-  folly::Future<folly::Unit> close(Context *ctx) override;
-  /** Close the dispatcher and the associated pipeline. */
-  folly::Future<folly::Unit> close() override;
-
- private:
-  void CloseAndCleanUpCalls();
-
- private:
-  std::recursive_mutex mutex_;
-  concurrent_map<uint32_t, folly::Promise<std::unique_ptr<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.
-  std::atomic<uint32_t> current_call_id_;
-  std::string server_;
-  bool is_closed_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/client-handler.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/client-handler.cc b/hbase-native-client/connection/client-handler.cc
deleted file mode 100644
index 983a68c..0000000
--- a/hbase-native-client/connection/client-handler.cc
+++ /dev/null
@@ -1,145 +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 "connection/client-handler.h"
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/Likely.h>
-#include <folly/io/async/AsyncSocketException.h>
-#include <glog/logging.h>
-#include <string>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "if/Client.pb.h"
-#include "if/RPC.pb.h"
-
-using google::protobuf::Message;
-
-namespace hbase {
-
-ClientHandler::ClientHandler(std::string user_name, std::shared_ptr<Codec> codec,
-                             std::shared_ptr<Configuration> conf, const std::string &server)
-    : user_name_(user_name),
-      serde_(codec),
-      conf_(conf),
-      server_(server),
-      once_flag_(std::make_unique<std::once_flag>()),
-      resp_msgs_(
-          std::make_unique<concurrent_map<uint32_t, std::shared_ptr<google::protobuf::Message>>>(
-              5000)) {}
-
-void ClientHandler::read(Context *ctx, std::unique_ptr<folly::IOBuf> buf) {
-  if (LIKELY(buf != nullptr)) {
-    buf->coalesce();
-    auto received = std::make_unique<Response>();
-    pb::ResponseHeader header;
-
-    int used_bytes = serde_.ParseDelimited(buf.get(), &header);
-    VLOG(3) << "Read RPC ResponseHeader size=" << used_bytes << " call_id=" << header.call_id()
-            << " has_exception=" << header.has_exception() << ", server: " << server_;
-
-    auto resp_msg = resp_msgs_->find_and_erase(header.call_id());
-
-    // set the call_id.
-    // This will be used to by the dispatcher to match up
-    // the promise with the response.
-    received->set_call_id(header.call_id());
-
-    // If there was an exception then there's no
-    // data left on the wire.
-    if (header.has_exception() == false) {
-      buf->trimStart(used_bytes);
-
-      int cell_block_length = 0;
-      used_bytes = serde_.ParseDelimited(buf.get(), resp_msg.get());
-      if (header.has_cell_block_meta() && header.cell_block_meta().has_length()) {
-        cell_block_length = header.cell_block_meta().length();
-      }
-
-      VLOG(3) << "Read RPCResponse, buf length:" << buf->length()
-              << ", header PB length:" << used_bytes << ", cell_block length:" << cell_block_length
-              << ", server: " << server_;
-
-      // Make sure that bytes were parsed.
-      CHECK((used_bytes + cell_block_length) == buf->length());
-
-      if (cell_block_length > 0) {
-        auto cell_scanner = serde_.CreateCellScanner(std::move(buf), used_bytes, cell_block_length);
-        received->set_cell_scanner(std::shared_ptr<CellScanner>{cell_scanner.release()});
-      }
-
-      received->set_resp_msg(resp_msg);
-    } else {
-      hbase::pb::ExceptionResponse exceptionResponse = header.exception();
-
-      std::string what;
-      std::string exception_class_name = exceptionResponse.has_exception_class_name()
-                                             ? exceptionResponse.exception_class_name()
-                                             : "";
-      std::string stack_trace =
-          exceptionResponse.has_stack_trace() ? exceptionResponse.stack_trace() : "";
-      what.append(stack_trace);
-
-      auto remote_exception = std::make_unique<RemoteException>(what);
-      remote_exception->set_exception_class_name(exception_class_name)
-          ->set_stack_trace(stack_trace)
-          ->set_hostname(exceptionResponse.has_hostname() ? exceptionResponse.hostname() : "")
-          ->set_port(exceptionResponse.has_port() ? exceptionResponse.port() : 0);
-      if (exceptionResponse.has_do_not_retry()) {
-        remote_exception->set_do_not_retry(exceptionResponse.do_not_retry());
-      }
-
-      VLOG(3) << "Exception RPC ResponseHeader, call_id=" << header.call_id()
-              << " exception.what=" << remote_exception->what()
-              << ", do_not_retry=" << remote_exception->do_not_retry() << ", server: " << server_;
-      received->set_exception(folly::exception_wrapper{*remote_exception});
-    }
-    ctx->fireRead(std::move(received));
-  }
-}
-
-folly::Future<folly::Unit> ClientHandler::write(Context *ctx, std::unique_ptr<Request> r) {
-  /* for RPC test, there's no need to send connection header */
-  if (!conf_->GetBool(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE,
-                      RpcSerde::DEFAULT_HBASE_CLIENT_RPC_TEST_MODE)) {
-    // We need to send the header once.
-    // So use call_once to make sure that only one thread wins this.
-    std::call_once((*once_flag_), [ctx, this]() {
-      VLOG(3) << "Writing RPC Header to server: " << server_;
-      auto header = serde_.Header(user_name_);
-      ctx->fireWrite(std::move(header));
-    });
-  }
-
-  VLOG(3) << "Writing RPC Request:" << r->DebugString() << ", server: " << server_;
-
-  // Now store the call id to response.
-  resp_msgs_->insert(std::make_pair(r->call_id(), r->resp_msg()));
-
-  try {
-    // Send the data down the pipeline.
-    return ctx->fireWrite(serde_.Request(r->call_id(), r->method(), r->req_msg().get()));
-  } catch (const folly::AsyncSocketException &e) {
-    /* clear protobuf::Message to avoid overflow. */
-    resp_msgs_->erase(r->call_id());
-    throw e;
-  }
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/client-handler.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/client-handler.h b/hbase-native-client/connection/client-handler.h
deleted file mode 100644
index b6f19a2..0000000
--- a/hbase-native-client/connection/client-handler.h
+++ /dev/null
@@ -1,89 +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.
- *
- */
-#pragma once
-
-#include <wangle/channel/Handler.h>
-
-#include <atomic>
-#include <memory>
-#include <mutex>
-#include <string>
-#include <utility>
-
-#include "core/configuration.h"
-#include "exceptions/exception.h"
-#include "serde/codec.h"
-#include "serde/rpc-serde.h"
-#include "utils/concurrent-map.h"
-
-// Forward decs.
-namespace hbase {
-class Request;
-class Response;
-class HeaderInfo;
-}
-namespace google {
-namespace protobuf {
-class Message;
-}
-}
-
-namespace hbase {
-
-/**
- * wangle::Handler implementation to convert hbase::Request to IOBuf and
- * convert IOBuf to hbase::Response.
- *
- * This class deals with sending the connection header and preamble
- * on first request.
- */
-class ClientHandler
-    : public wangle::Handler<std::unique_ptr<folly::IOBuf>, std::unique_ptr<Response>,
-                             std::unique_ptr<Request>, std::unique_ptr<folly::IOBuf>> {
- public:
-  /**
-   * Create the handler
-   * @param user_name the user name of the user running this process.
-   */
-  ClientHandler(std::string user_name, std::shared_ptr<Codec> codec,
-                std::shared_ptr<Configuration> conf, const std::string &server);
-
-  /**
-   * Get bytes from the wire.
-   * This should be the full message as the length field decoder should be
-   * in the pipeline before this.
-   */
-  void read(Context *ctx, std::unique_ptr<folly::IOBuf> msg) override;
-
-  /**
-   * Write the data down the wire.
-   */
-  folly::Future<folly::Unit> write(Context *ctx, std::unique_ptr<Request> r) override;
-
- private:
-  std::unique_ptr<std::once_flag> once_flag_;
-  std::string user_name_;
-  RpcSerde serde_;
-  std::string server_;  // for logging
-  std::shared_ptr<Configuration> conf_;
-
-  // in flight requests
-  std::unique_ptr<concurrent_map<uint32_t, std::shared_ptr<google::protobuf::Message>>> resp_msgs_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-factory.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-factory.cc b/hbase-native-client/connection/connection-factory.cc
deleted file mode 100644
index 751073e..0000000
--- a/hbase-native-client/connection/connection-factory.cc
+++ /dev/null
@@ -1,88 +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 <folly/Conv.h>
-#include <glog/logging.h>
-#include <wangle/channel/Handler.h>
-
-#include <chrono>
-
-#include <folly/ExceptionWrapper.h>
-#include <folly/SocketAddress.h>
-#include <folly/io/async/AsyncSocketException.h>
-
-#include "connection/client-dispatcher.h"
-#include "connection/connection-factory.h"
-#include "connection/pipeline.h"
-#include "connection/sasl-handler.h"
-#include "connection/service.h"
-#include "exceptions/exception.h"
-
-using std::chrono::milliseconds;
-using std::chrono::nanoseconds;
-
-namespace hbase {
-
-ConnectionFactory::ConnectionFactory(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                                     std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                                     std::shared_ptr<Codec> codec,
-                                     std::shared_ptr<Configuration> conf,
-                                     nanoseconds connect_timeout)
-    : connect_timeout_(connect_timeout),
-      io_executor_(io_executor),
-      cpu_executor_(cpu_executor),
-      conf_(conf),
-      pipeline_factory_(std::make_shared<RpcPipelineFactory>(codec, conf)) {}
-
-std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> ConnectionFactory::MakeBootstrap() {
-  auto client = std::make_shared<wangle::ClientBootstrap<SerializePipeline>>();
-  client->group(io_executor_);
-  client->pipelineFactory(pipeline_factory_);
-
-  // TODO: Opened https://github.com/facebook/wangle/issues/85 in wangle so that we can set socket
-  //  options like TCP_NODELAY, SO_KEEPALIVE, CONNECT_TIMEOUT_MILLIS, etc.
-
-  return client;
-}
-
-std::shared_ptr<HBaseService> ConnectionFactory::Connect(
-    std::shared_ptr<RpcConnection> rpc_connection,
-    std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap,
-    const std::string &hostname, uint16_t port) {
-  // connection should happen from an IO thread
-  try {
-    auto future = via(io_executor_.get()).then([=]() {
-      VLOG(1) << "Connecting to server: " << hostname << ":" << port;
-      return client_bootstrap->connect(folly::SocketAddress(hostname, port, true),
-                                       std::chrono::duration_cast<milliseconds>(connect_timeout_));
-    });
-
-    // See about using shared promise for this.
-    auto pipeline = future.get();
-
-    VLOG(1) << "Connected to server: " << hostname << ":" << port;
-    auto dispatcher =
-        std::make_shared<ClientDispatcher>(hostname + ":" + folly::to<std::string>(port));
-    dispatcher->setPipeline(pipeline);
-    return dispatcher;
-  } catch (const folly::AsyncSocketException &e) {
-    throw ConnectionException(folly::exception_wrapper{e});
-  }
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-factory.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-factory.h b/hbase-native-client/connection/connection-factory.h
deleted file mode 100644
index c4e63c2..0000000
--- a/hbase-native-client/connection/connection-factory.h
+++ /dev/null
@@ -1,83 +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.
- *
- */
-#pragma once
-
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
-#include <wangle/service/Service.h>
-
-#include <chrono>
-#include <memory>
-#include <string>
-
-#include "connection/pipeline.h"
-#include "connection/request.h"
-#include "connection/response.h"
-#include "connection/service.h"
-#include "security/user.h"
-
-namespace hbase {
-
-class RpcConnection;
-
-/**
- * Class to create a ClientBootstrap and turn it into a connected
- * pipeline.
- */
-class ConnectionFactory {
- public:
-  /**
-   * Constructor.
-   * There should only be one ConnectionFactory per client.
-   */
-  ConnectionFactory(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                    std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                    std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
-                    std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
-
-  /** Default Destructor */
-  virtual ~ConnectionFactory() = default;
-
-  /**
-   * Create a BootStrap from which a connection can be made.
-   */
-  virtual std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> MakeBootstrap();
-
-  /**
-   * Connect a ClientBootstrap to a server and return the pipeline.
-   *
-   * This is mostly visible so that mocks can override socket connections.
-   */
-  virtual std::shared_ptr<HBaseService> Connect(
-      std::shared_ptr<RpcConnection> rpc_connection,
-      std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>> client_bootstrap,
-      const std::string &hostname, uint16_t port);
-
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor() { return io_executor_; }
-
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor() { return cpu_executor_; }
-
- private:
-  std::chrono::nanoseconds connect_timeout_;
-  std::shared_ptr<Configuration> conf_;
-  std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor_;
-  std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor_;
-  std::shared_ptr<RpcPipelineFactory> pipeline_factory_;
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-id.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-id.h b/hbase-native-client/connection/connection-id.h
deleted file mode 100644
index 065b484..0000000
--- a/hbase-native-client/connection/connection-id.h
+++ /dev/null
@@ -1,93 +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.
- *
- */
-#pragma once
-
-#include <boost/functional/hash.hpp>
-
-#include <memory>
-#include <string>
-#include <utility>
-
-#include "if/HBase.pb.h"
-#include "security/user.h"
-
-namespace hbase {
-
-class ConnectionId {
- public:
-  ConnectionId(const std::string &host, uint16_t port)
-      : ConnectionId(host, port, security::User::defaultUser(), "") {}
-
-  ConnectionId(const std::string &host, uint16_t port, std::shared_ptr<security::User> user)
-      : ConnectionId(host, port, user, "") {}
-
-  ConnectionId(const std::string &host, uint16_t port, std::shared_ptr<security::User> user,
-               const std::string &service_name)
-      : user_(user), service_name_(service_name), host_(host), port_(port) {}
-
-  ConnectionId(const std::string &host, uint16_t port, const std::string &service_name)
-      : user_(security::User::defaultUser()),
-        service_name_(service_name),
-        host_(host),
-        port_(port) {}
-
-  virtual ~ConnectionId() = default;
-
-  std::shared_ptr<security::User> user() const { return user_; }
-  std::string service_name() const { return service_name_; }
-  std::string host() { return host_; }
-  uint16_t port() { return port_; }
-
- private:
-  std::shared_ptr<security::User> user_;
-  std::string service_name_;
-  std::string host_;
-  uint16_t port_;
-};
-
-/* Equals function for ConnectionId */
-struct ConnectionIdEquals {
-  /** equals */
-  bool operator()(const std::shared_ptr<ConnectionId> &lhs,
-                  const std::shared_ptr<ConnectionId> &rhs) const {
-    return userEquals(lhs->user(), rhs->user()) && lhs->host() == rhs->host() &&
-           lhs->port() == rhs->port() && lhs->service_name() == rhs->service_name();
-  }
-
- private:
-  bool userEquals(const std::shared_ptr<security::User> &lhs,
-                  const std::shared_ptr<security::User> &rhs) const {
-    return lhs == nullptr ? rhs == nullptr
-                          : (rhs == nullptr ? false : lhs->user_name() == rhs->user_name());
-  }
-};
-
-/** Hash for ConnectionId. */
-struct ConnectionIdHash {
-  /** hash */
-  std::size_t operator()(const std::shared_ptr<ConnectionId> &ci) const {
-    std::size_t h = 0;
-    boost::hash_combine(h, ci->user() == nullptr ? 0 : ci->user()->user_name());
-    boost::hash_combine(h, ci->host());
-    boost::hash_combine(h, ci->port());
-    boost::hash_combine(h, ci->service_name());
-    return h;
-  }
-};
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-pool-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-pool-test.cc b/hbase-native-client/connection/connection-pool-test.cc
deleted file mode 100644
index 0dc8e14..0000000
--- a/hbase-native-client/connection/connection-pool-test.cc
+++ /dev/null
@@ -1,140 +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 <folly/Logging.h>
-#include <gmock/gmock.h>
-
-#include "connection/connection-factory.h"
-#include "connection/connection-id.h"
-#include "connection/connection-pool.h"
-#include "if/HBase.pb.h"
-#include "serde/server-name.h"
-
-using hbase::pb::ServerName;
-using ::testing::Return;
-using ::testing::_;
-using hbase::ConnectionFactory;
-using hbase::ConnectionPool;
-using hbase::ConnectionId;
-using hbase::HBaseService;
-using hbase::Request;
-using hbase::Response;
-using hbase::RpcConnection;
-using hbase::SerializePipeline;
-
-class MockConnectionFactory : public ConnectionFactory {
- public:
-  MockConnectionFactory() : ConnectionFactory(nullptr, nullptr, nullptr, nullptr) {}
-  MOCK_METHOD0(MakeBootstrap, std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>>());
-  MOCK_METHOD4(Connect, std::shared_ptr<HBaseService>(
-                            std::shared_ptr<RpcConnection> rpc_connection,
-                            std::shared_ptr<wangle::ClientBootstrap<SerializePipeline>>,
-                            const std::string &hostname, uint16_t port));
-};
-
-class MockBootstrap : public wangle::ClientBootstrap<SerializePipeline> {};
-
-class MockService : public HBaseService {
- public:
-  folly::Future<std::unique_ptr<Response>> operator()(std::unique_ptr<Request> req) override {
-    return folly::makeFuture<std::unique_ptr<Response>>(
-        std::make_unique<Response>(do_operation(req.get())));
-  }
-  MOCK_METHOD1(do_operation, Response(Request *));
-};
-
-TEST(TestConnectionPool, TestOnlyCreateOnce) {
-  auto hostname = std::string{"hostname"};
-  auto mock_boot = std::make_shared<MockBootstrap>();
-  auto mock_service = std::make_shared<MockService>();
-  auto mock_cf = std::make_shared<MockConnectionFactory>();
-  uint32_t port{999};
-
-  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(1).WillRepeatedly(Return(mock_service));
-  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(1).WillRepeatedly(Return(mock_boot));
-  EXPECT_CALL((*mock_service), do_operation(_)).Times(1).WillRepeatedly(Return(Response{}));
-  ConnectionPool cp{mock_cf};
-
-  auto remote_id = std::make_shared<ConnectionId>(hostname, port);
-  auto result = cp.GetConnection(remote_id);
-  ASSERT_TRUE(result != nullptr);
-  result = cp.GetConnection(remote_id);
-  result->SendRequest(nullptr);
-}
-
-TEST(TestConnectionPool, TestOnlyCreateMultipleDispose) {
-  std::string hostname_one{"hostname"};
-  std::string hostname_two{"hostname_two"};
-  uint32_t port{999};
-
-  auto mock_boot = std::make_shared<MockBootstrap>();
-  auto mock_service = std::make_shared<MockService>();
-  auto mock_cf = std::make_shared<MockConnectionFactory>();
-
-  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(2).WillRepeatedly(Return(mock_service));
-  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(2).WillRepeatedly(Return(mock_boot));
-  EXPECT_CALL((*mock_service), do_operation(_)).Times(4).WillRepeatedly(Return(Response{}));
-  ConnectionPool cp{mock_cf};
-
-  {
-    auto remote_id = std::make_shared<ConnectionId>(hostname_one, port);
-    auto result_one = cp.GetConnection(remote_id);
-    result_one->SendRequest(nullptr);
-    auto remote_id2 = std::make_shared<ConnectionId>(hostname_two, port);
-    auto result_two = cp.GetConnection(remote_id2);
-    result_two->SendRequest(nullptr);
-  }
-  auto remote_id = std::make_shared<ConnectionId>(hostname_one, port);
-  auto result_one = cp.GetConnection(remote_id);
-  result_one->SendRequest(nullptr);
-  auto remote_id2 = std::make_shared<ConnectionId>(hostname_two, port);
-  auto result_two = cp.GetConnection(remote_id2);
-  result_two->SendRequest(nullptr);
-}
-
-TEST(TestConnectionPool, TestCreateOneConnectionForOneService) {
-  std::string hostname{"hostname"};
-  uint32_t port{999};
-  std::string service1{"service1"};
-  std::string service2{"service2"};
-
-  auto mock_boot = std::make_shared<MockBootstrap>();
-  auto mock_service = std::make_shared<MockService>();
-  auto mock_cf = std::make_shared<MockConnectionFactory>();
-
-  EXPECT_CALL((*mock_cf), Connect(_, _, _, _)).Times(2).WillRepeatedly(Return(mock_service));
-  EXPECT_CALL((*mock_cf), MakeBootstrap()).Times(2).WillRepeatedly(Return(mock_boot));
-  EXPECT_CALL((*mock_service), do_operation(_)).Times(4).WillRepeatedly(Return(Response{}));
-  ConnectionPool cp{mock_cf};
-
-  {
-    auto remote_id = std::make_shared<ConnectionId>(hostname, port, service1);
-    auto result_one = cp.GetConnection(remote_id);
-    result_one->SendRequest(nullptr);
-    auto remote_id2 = std::make_shared<ConnectionId>(hostname, port, service2);
-    auto result_two = cp.GetConnection(remote_id2);
-    result_two->SendRequest(nullptr);
-  }
-  auto remote_id = std::make_shared<ConnectionId>(hostname, port, service1);
-  auto result_one = cp.GetConnection(remote_id);
-  result_one->SendRequest(nullptr);
-  auto remote_id2 = std::make_shared<ConnectionId>(hostname, port, service2);
-  auto result_two = cp.GetConnection(remote_id2);
-  result_two->SendRequest(nullptr);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-pool.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-pool.cc b/hbase-native-client/connection/connection-pool.cc
deleted file mode 100644
index e1f6358..0000000
--- a/hbase-native-client/connection/connection-pool.cc
+++ /dev/null
@@ -1,119 +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 "connection/connection-pool.h"
-
-#include <folly/Conv.h>
-#include <folly/Logging.h>
-#include <wangle/service/Service.h>
-
-#include <memory>
-#include <string>
-#include <utility>
-
-using std::chrono::nanoseconds;
-
-namespace hbase {
-
-ConnectionPool::ConnectionPool(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                               std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                               std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
-                               nanoseconds connect_timeout)
-    : cf_(std::make_shared<ConnectionFactory>(io_executor, cpu_executor, codec, conf,
-                                              connect_timeout)),
-      connections_(),
-      map_mutex_(),
-      conf_(conf) {}
-ConnectionPool::ConnectionPool(std::shared_ptr<ConnectionFactory> cf)
-    : cf_(cf), connections_(), map_mutex_() {}
-
-ConnectionPool::~ConnectionPool() {}
-
-std::shared_ptr<RpcConnection> ConnectionPool::GetConnection(
-    std::shared_ptr<ConnectionId> remote_id) {
-  // Try and get th cached connection.
-  auto found_ptr = GetCachedConnection(remote_id);
-
-  // If there's no connection then create it.
-  if (found_ptr == nullptr) {
-    found_ptr = GetNewConnection(remote_id);
-  }
-  return found_ptr;
-}
-
-std::shared_ptr<RpcConnection> ConnectionPool::GetCachedConnection(
-    std::shared_ptr<ConnectionId> remote_id) {
-  folly::SharedMutexWritePriority::ReadHolder holder(map_mutex_);
-  auto found = connections_.find(remote_id);
-  if (found == connections_.end()) {
-    return nullptr;
-  }
-  return found->second;
-}
-
-std::shared_ptr<RpcConnection> ConnectionPool::GetNewConnection(
-    std::shared_ptr<ConnectionId> remote_id) {
-  // Grab the upgrade lock. While we are double checking other readers can
-  // continue on
-  folly::SharedMutexWritePriority::UpgradeHolder u_holder{map_mutex_};
-
-  // Now check if someone else created the connection before we got the lock
-  // This is safe since we hold the upgrade lock.
-  // upgrade lock is more power than the reader lock.
-  auto found = connections_.find(remote_id);
-  if (found != connections_.end() && found->second != nullptr) {
-    return found->second;
-  } else {
-    // Yeah it looks a lot like there's no connection
-    folly::SharedMutexWritePriority::WriteHolder w_holder{std::move(u_holder)};
-
-    // Make double sure there are not stale connections hanging around.
-    connections_.erase(remote_id);
-
-    /* create new connection */
-    auto connection = std::make_shared<RpcConnection>(remote_id, cf_);
-
-    connections_.insert(std::make_pair(remote_id, connection));
-
-    return connection;
-  }
-}
-
-void ConnectionPool::Close(std::shared_ptr<ConnectionId> remote_id) {
-  folly::SharedMutexWritePriority::WriteHolder holder{map_mutex_};
-  DLOG(INFO) << "Closing RPC Connection to host:" << remote_id->host()
-             << ", port:" << folly::to<std::string>(remote_id->port());
-
-  auto found = connections_.find(remote_id);
-  if (found == connections_.end() || found->second == nullptr) {
-    return;
-  }
-  found->second->Close();
-  connections_.erase(found);
-}
-
-void ConnectionPool::Close() {
-  folly::SharedMutexWritePriority::WriteHolder holder{map_mutex_};
-  for (auto &item : connections_) {
-    auto &con = item.second;
-    con->Close();
-  }
-  connections_.clear();
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/connection-pool.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/connection-pool.h b/hbase-native-client/connection/connection-pool.h
deleted file mode 100644
index 9af1e7f..0000000
--- a/hbase-native-client/connection/connection-pool.h
+++ /dev/null
@@ -1,90 +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.
- *
- */
-#pragma once
-
-#include <folly/SharedMutex.h>
-#include <boost/functional/hash.hpp>
-#include <chrono>
-#include <memory>
-#include <mutex>
-#include <unordered_map>
-
-#include "connection/connection-factory.h"
-#include "connection/connection-id.h"
-#include "connection/rpc-connection.h"
-#include "connection/service.h"
-#include "if/HBase.pb.h"
-
-namespace hbase {
-
-/**
- * @brief Connection pooling for HBase rpc connection.
- *
- * This is a thread safe connection pool. It allows getting
- * a shared rpc connection to HBase servers by connection id.
- */
-class ConnectionPool {
- public:
-  /** Create connection pool wit default connection factory */
-  ConnectionPool(std::shared_ptr<wangle::IOThreadPoolExecutor> io_executor,
-                 std::shared_ptr<wangle::CPUThreadPoolExecutor> cpu_executor,
-                 std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf,
-                 std::chrono::nanoseconds connect_timeout = std::chrono::nanoseconds(0));
-
-  /**
-   * Constructor that allows specifiying the connetion factory.
-   * This is useful for testing.
-   */
-  explicit ConnectionPool(std::shared_ptr<ConnectionFactory> cf);
-
-  /**
-   * Destructor.
-   * All connections will be close.
-   * All connections will be released
-   */
-  ~ConnectionPool();
-
-  /**
-   * Get a connection to the server name. Start time is ignored.
-   * This can be a blocking operation for a short time.
-   */
-  std::shared_ptr<RpcConnection> GetConnection(std::shared_ptr<ConnectionId> remote_id);
-
-  /**
-   * Close/remove a connection.
-   */
-  void Close(std::shared_ptr<ConnectionId> remote_id);
-
-  /**
-   * Close the Connection Pool
-   */
-  void Close();
-
- private:
-  std::shared_ptr<RpcConnection> GetCachedConnection(std::shared_ptr<ConnectionId> remote_id);
-  std::shared_ptr<RpcConnection> GetNewConnection(std::shared_ptr<ConnectionId> remote_id);
-  std::unordered_map<std::shared_ptr<ConnectionId>, std::shared_ptr<RpcConnection>,
-                     ConnectionIdHash, ConnectionIdEquals>
-      connections_;
-  folly::SharedMutexWritePriority map_mutex_;
-  std::shared_ptr<ConnectionFactory> cf_;
-  std::shared_ptr<Configuration> conf_;
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/pipeline.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/pipeline.cc b/hbase-native-client/connection/pipeline.cc
deleted file mode 100644
index 9c790b6..0000000
--- a/hbase-native-client/connection/pipeline.cc
+++ /dev/null
@@ -1,55 +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 "connection/pipeline.h"
-
-#include <folly/Logging.h>
-#include <wangle/channel/AsyncSocketHandler.h>
-#include <wangle/channel/EventBaseHandler.h>
-#include <wangle/channel/OutputBufferingHandler.h>
-#include <wangle/codec/LengthFieldBasedFrameDecoder.h>
-
-#include "connection/client-handler.h"
-#include "connection/sasl-handler.h"
-
-namespace hbase {
-
-RpcPipelineFactory::RpcPipelineFactory(std::shared_ptr<Codec> codec,
-                                       std::shared_ptr<Configuration> conf)
-    : user_util_(), codec_(codec), conf_(conf) {}
-SerializePipeline::Ptr RpcPipelineFactory::newPipeline(
-    std::shared_ptr<folly::AsyncTransportWrapper> sock) {
-  folly::SocketAddress addr;  // for logging
-  sock->getPeerAddress(&addr);
-
-  auto pipeline = SerializePipeline::create();
-  pipeline->addBack(wangle::AsyncSocketHandler{sock});
-  pipeline->addBack(wangle::EventBaseHandler{});
-  bool secure = false;
-  /* for RPC test, there's no need to setup Sasl */
-  if (!conf_->GetBool(RpcSerde::HBASE_CLIENT_RPC_TEST_MODE,
-                      RpcSerde::DEFAULT_HBASE_CLIENT_RPC_TEST_MODE)) {
-    secure = security::User::IsSecurityEnabled(*conf_);
-    pipeline->addBack(SaslHandler{user_util_.user_name(secure), conf_});
-  }
-  pipeline->addBack(wangle::LengthFieldBasedFrameDecoder{});
-  pipeline->addBack(ClientHandler{user_util_.user_name(secure), codec_, conf_, addr.describe()});
-  pipeline->finalize();
-  return pipeline;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/connection/pipeline.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/connection/pipeline.h b/hbase-native-client/connection/pipeline.h
deleted file mode 100644
index add7fe5..0000000
--- a/hbase-native-client/connection/pipeline.h
+++ /dev/null
@@ -1,63 +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.
- *
- */
-#pragma once
-
-#include <folly/io/IOBufQueue.h>
-#include <wangle/service/Service.h>
-
-#include <memory>
-
-#include "connection/request.h"
-#include "connection/response.h"
-#include "core/configuration.h"
-#include "serde/codec.h"
-#include "utils/user-util.h"
-
-namespace hbase {
-
-/** Pipeline to turn IOBuf into requests */
-using SerializePipeline = wangle::Pipeline<folly::IOBufQueue &, std::unique_ptr<Request>>;
-
-/**
- * Factory to create new pipelines for HBase RPC's.
- */
-class RpcPipelineFactory : public wangle::PipelineFactory<SerializePipeline> {
- public:
-  /**
-   * Constructor. This will create user util.
-   */
-  explicit RpcPipelineFactory(std::shared_ptr<Codec> codec, std::shared_ptr<Configuration> conf);
-
-  /**
-   * Create a new pipeline.
-   * The pipeline will be:
-   *
-   * - Async Socke Handler
-   * - Event Base Handler
-   * - Length Field Based Frame Decoder
-   * - Client Handler
-   */
-  SerializePipeline::Ptr newPipeline(std::shared_ptr<folly::AsyncTransportWrapper> sock) override;
-
- private:
-  UserUtil user_util_;
-  std::shared_ptr<Codec> codec_;
-  std::shared_ptr<Configuration> conf_;
-};
-}  // namespace hbase


[03/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Comparator.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Comparator.proto b/hbase-native-client/src/hbase/if/Comparator.proto
new file mode 100644
index 0000000..496b68d
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Comparator.proto
@@ -0,0 +1,74 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for filters
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ComparatorProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+// This file contains protocol buffers that are used for comparators (e.g. in filters)
+
+message Comparator {
+  required string name = 1;
+  optional bytes serialized_comparator = 2;
+}
+
+message ByteArrayComparable {
+  optional bytes value = 1;
+}
+
+message BinaryComparator {
+  required ByteArrayComparable comparable = 1;
+}
+
+message LongComparator {
+  required ByteArrayComparable comparable = 1;
+}
+
+message BinaryPrefixComparator {
+  required ByteArrayComparable comparable = 1;
+}
+
+message BitComparator {
+  required ByteArrayComparable comparable = 1;
+  required BitwiseOp bitwise_op = 2;
+
+  enum BitwiseOp {
+    AND = 1;
+    OR = 2;
+    XOR = 3;
+  }
+}
+
+message NullComparator {
+}
+
+message RegexStringComparator {
+  required string pattern = 1;
+  required int32 pattern_flags = 2;
+  required string charset = 3;
+  optional string engine = 4;
+}
+
+message SubstringComparator {
+  required string substr = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Encryption.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Encryption.proto b/hbase-native-client/src/hbase/if/Encryption.proto
new file mode 100644
index 0000000..97ab5b2
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Encryption.proto
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers used for encryption
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "EncryptionProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message WrappedKey {
+  required string algorithm = 1;
+  required uint32 length = 2;
+  required bytes data = 3;
+  optional bytes iv = 4;
+  optional bytes hash = 5;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/ErrorHandling.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/ErrorHandling.proto b/hbase-native-client/src/hbase/if/ErrorHandling.proto
new file mode 100644
index 0000000..be9a743
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/ErrorHandling.proto
@@ -0,0 +1,58 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for error handling
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ErrorHandlingProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+/**
+ * Protobuf version of a java.lang.StackTraceElement
+ * so we can serialize exceptions.
+ */
+message StackTraceElementMessage {
+  optional string declaring_class = 1;
+  optional string method_name = 2;
+  optional string file_name = 3;
+  optional int32 line_number = 4;
+}
+
+/**
+ * Cause of a remote failure for a generic exception. Contains
+ * all the information for a generic exception as well as
+ * optional info about the error for generic info passing
+ * (which should be another protobuffed class).
+ */
+message GenericExceptionMessage {
+  optional string class_name = 1;
+  optional string message = 2;
+  optional bytes error_info = 3;
+  repeated StackTraceElementMessage trace = 4;
+}
+
+/**
+ * Exception sent across the wire when a remote task needs
+ * to notify other tasks that it failed and why
+ */
+message ForeignExceptionMessage {
+  optional string source = 1;
+  optional GenericExceptionMessage generic_exception = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/FS.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/FS.proto b/hbase-native-client/src/hbase/if/FS.proto
new file mode 100644
index 0000000..9e93120
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/FS.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are written into the filesystem
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "FSProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+/**
+ * The ${HBASE_ROOTDIR}/hbase.version file content
+ */
+message HBaseVersionFileContent {
+  required string version = 1;
+}
+
+/**
+ * Reference file content used when we split an hfile under a region.
+ */
+message Reference {
+  required bytes splitkey = 1;
+  enum Range {
+    TOP = 0;
+    BOTTOM = 1;
+  }
+  required Range range = 2;
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Filter.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Filter.proto b/hbase-native-client/src/hbase/if/Filter.proto
new file mode 100644
index 0000000..1fa6697
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Filter.proto
@@ -0,0 +1,171 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for filters
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "FilterProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Comparator.proto";
+
+message Filter {
+  required string name = 1;
+  optional bytes serialized_filter = 2;
+}
+
+message ColumnCountGetFilter {
+  required int32 limit = 1;
+}
+
+message ColumnPaginationFilter {
+  required int32 limit = 1;
+  optional int32 offset = 2;
+  optional bytes column_offset = 3;
+}
+
+message ColumnPrefixFilter {
+  required bytes prefix = 1;
+}
+
+message ColumnRangeFilter {
+  optional bytes min_column = 1;
+  optional bool min_column_inclusive = 2;
+  optional bytes max_column = 3;
+  optional bool max_column_inclusive = 4;
+}
+
+message CompareFilter {
+  required CompareType compare_op = 1;
+  optional Comparator comparator = 2;
+}
+
+message DependentColumnFilter {
+  required CompareFilter compare_filter = 1;
+  optional bytes column_family = 2;
+  optional bytes column_qualifier = 3;
+  optional bool drop_dependent_column = 4;
+}
+
+message FamilyFilter {
+  required CompareFilter compare_filter = 1;
+}
+
+message FilterList {
+  required Operator operator = 1;
+  repeated Filter filters = 2;
+
+  enum Operator {
+    MUST_PASS_ALL = 1;
+    MUST_PASS_ONE = 2;
+  }
+}
+
+message FilterWrapper {
+  required Filter filter = 1;
+}
+
+message FirstKeyOnlyFilter {
+}
+
+message FirstKeyValueMatchingQualifiersFilter {
+  repeated bytes qualifiers = 1;
+}
+
+message FuzzyRowFilter {
+  repeated BytesBytesPair fuzzy_keys_data = 1;
+}
+
+message InclusiveStopFilter {
+  optional bytes stop_row_key = 1;
+}
+
+message KeyOnlyFilter {
+  required bool len_as_val = 1;
+}
+
+message MultipleColumnPrefixFilter {
+  repeated bytes sorted_prefixes = 1;
+}
+
+message PageFilter {
+  required int64 page_size = 1;
+}
+
+message PrefixFilter {
+  optional bytes prefix = 1;
+}
+
+message QualifierFilter {
+  required CompareFilter compare_filter = 1;
+}
+
+message RandomRowFilter {
+  required float chance = 1;
+}
+
+message RowFilter {
+  required CompareFilter compare_filter = 1;
+}
+
+message SingleColumnValueExcludeFilter {
+  required SingleColumnValueFilter single_column_value_filter = 1;
+}
+
+message SingleColumnValueFilter {
+  optional bytes column_family = 1;
+  optional bytes column_qualifier = 2;
+  required CompareType compare_op = 3;
+  required Comparator comparator = 4;
+  optional bool filter_if_missing = 5;
+  optional bool latest_version_only = 6;
+}
+
+message SkipFilter {
+  required Filter filter = 1;
+}
+
+message TimestampsFilter {
+  repeated int64 timestamps = 1 [packed=true];
+  optional bool can_hint = 2;
+}
+
+message ValueFilter {
+  required CompareFilter compare_filter = 1;
+}
+
+message WhileMatchFilter {
+  required Filter filter = 1;
+}
+message FilterAllFilter {
+}
+
+message RowRange {
+  optional bytes start_row = 1;
+  optional bool start_row_inclusive = 2;
+  optional bytes stop_row = 3;
+  optional bool stop_row_inclusive =4;
+}
+
+message MultiRowRangeFilter {
+  repeated RowRange row_range_list = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/HBase.proto b/hbase-native-client/src/hbase/if/HBase.proto
new file mode 100644
index 0000000..c36b214
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/HBase.proto
@@ -0,0 +1,253 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are shared throughout HBase
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "HBaseProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "Cell.proto";
+
+/**
+ * Table Name
+ */
+message TableName {
+  required bytes namespace = 1;
+  required bytes qualifier = 2;
+}
+
+/**
+ * Table Schema
+ * Inspired by the rest TableSchema
+ */
+message TableSchema {
+  optional TableName table_name = 1;
+  repeated BytesBytesPair attributes = 2;
+  repeated ColumnFamilySchema column_families = 3;
+  repeated NameStringPair configuration = 4;
+}
+
+/** Denotes state of the table */
+message TableState {
+  // Table's current state
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+    DISABLING = 2;
+    ENABLING = 3;
+  }
+  // This is the table's state.
+  required State state = 1;
+}
+
+/**
+ * Column Family Schema
+ * Inspired by the rest ColumSchemaMessage
+ */
+message ColumnFamilySchema {
+  required bytes name = 1;
+  repeated BytesBytesPair attributes = 2;
+  repeated NameStringPair configuration = 3;
+}
+
+/**
+ * Protocol buffer version of HRegionInfo.
+ */
+message RegionInfo {
+  required uint64 region_id = 1;
+  required TableName table_name = 2;
+  optional bytes start_key = 3;
+  optional bytes end_key = 4;
+  optional bool offline = 5;
+  optional bool split = 6;
+  optional int32 replica_id = 7 [default = 0];
+}
+
+/**
+ * Protocol buffer for favored nodes
+ */
+message FavoredNodes {
+  repeated ServerName favored_node = 1;
+}
+
+/**
+ * Container protocol buffer to specify a region.
+ * You can specify region by region name, or the hash
+ * of the region name, which is known as encoded
+ * region name.
+ */
+message RegionSpecifier {
+  required RegionSpecifierType type = 1;
+  required bytes value = 2;
+
+  enum RegionSpecifierType {
+    // <tablename>,<startkey>,<regionId>.<encodedName>
+    REGION_NAME = 1;
+
+    // hash of <tablename>,<startkey>,<regionId>
+    ENCODED_REGION_NAME = 2;
+  }
+}
+
+/**
+ * A range of time. Both from and to are Java time
+ * stamp in milliseconds. If you don't specify a time
+ * range, it means all time.  By default, if not
+ * specified, from = 0, and to = Long.MAX_VALUE
+ */
+message TimeRange {
+  optional uint64 from = 1;
+  optional uint64 to = 2;
+}
+
+/* ColumnFamily Specific TimeRange */
+message ColumnFamilyTimeRange {
+  required bytes column_family = 1;
+  required TimeRange time_range = 2;
+}
+
+/* Comparison operators */
+enum CompareType {
+  LESS = 0;
+  LESS_OR_EQUAL = 1;
+  EQUAL = 2;
+  NOT_EQUAL = 3;
+  GREATER_OR_EQUAL = 4;
+  GREATER = 5;
+  NO_OP = 6;
+}
+
+/**
+ * Protocol buffer version of ServerName
+ */
+message ServerName {
+  required string host_name = 1;
+  optional uint32 port = 2;
+  optional uint64 start_code = 3;
+}
+
+// Comment data structures
+
+message Coprocessor {
+  required string name = 1;
+}
+
+message NameStringPair {
+  required string name = 1;
+  required string value = 2;
+}
+
+message NameBytesPair {
+  required string name = 1;
+  optional bytes value = 2;
+}
+
+message BytesBytesPair {
+  required bytes first = 1;
+  required bytes second = 2;
+}
+
+message NameInt64Pair {
+  optional string name = 1;
+  optional int64 value = 2;
+}
+
+/**
+ * Description of the snapshot to take
+ */
+message SnapshotDescription {
+  required string name = 1;
+  optional string table = 2; // not needed for delete, but checked for in taking snapshot
+  optional int64 creation_time = 3 [default = 0];
+  enum Type {
+    DISABLED = 0;
+    FLUSH = 1;
+    SKIPFLUSH = 2;
+  }
+  optional Type type = 4 [default = FLUSH];
+  optional int32 version = 5;
+  optional string owner = 6;
+}
+
+/**
+ * Description of the distributed procedure to take
+ */
+message ProcedureDescription {
+  required string signature = 1; // the unique signature of the procedure
+  optional string instance = 2; // the procedure instance name
+  optional int64 creation_time = 3 [default = 0];
+  repeated NameStringPair configuration = 4;
+}
+
+message EmptyMsg {
+}
+
+enum TimeUnit {
+  NANOSECONDS = 1;
+  MICROSECONDS = 2;
+  MILLISECONDS = 3;
+  SECONDS = 4;
+  MINUTES = 5;
+  HOURS = 6;
+  DAYS = 7;
+}
+
+message LongMsg {
+  required int64 long_msg = 1;
+}
+
+message DoubleMsg {
+  required double double_msg = 1;
+}
+
+message BigDecimalMsg {
+  required bytes bigdecimal_msg = 1;
+}
+
+message UUID {
+  required uint64 least_sig_bits = 1;
+  required uint64 most_sig_bits = 2;
+}
+
+message NamespaceDescriptor {
+  required bytes name = 1;
+  repeated NameStringPair configuration = 2;
+}
+
+// Rpc client version info proto. Included in ConnectionHeader on connection setup
+message VersionInfo {
+  required string version = 1;
+  required string url = 2;
+  required string revision = 3;
+  required string user = 4;
+  required string date = 5;
+  required string src_checksum = 6;
+  optional uint32 version_major = 7;
+  optional uint32 version_minor = 8;
+}
+
+/**
+ * Description of the region server info
+ */
+message RegionServerInfo {
+  optional int32 infoPort = 1;
+  optional VersionInfo version_info = 2;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/HFile.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/HFile.proto b/hbase-native-client/src/hbase/if/HFile.proto
new file mode 100644
index 0000000..5c5e4f3
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/HFile.proto
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "HFileProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+// Map of name/values
+message FileInfoProto {
+  repeated BytesBytesPair map_entry = 1;
+}
+
+// HFile file trailer
+message FileTrailerProto {
+  optional uint64 file_info_offset = 1;
+  optional uint64 load_on_open_data_offset = 2;
+  optional uint64 uncompressed_data_index_size = 3;
+  optional uint64 total_uncompressed_bytes = 4;
+  optional uint32 data_index_count = 5;
+  optional uint32 meta_index_count = 6;
+  optional uint64 entry_count = 7;
+  optional uint32 num_data_index_levels = 8;
+  optional uint64 first_data_block_offset = 9;
+  optional uint64 last_data_block_offset = 10;
+  optional string comparator_class_name = 11;
+  optional uint32 compression_codec = 12;
+  optional bytes encryption_key = 13;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/LoadBalancer.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/LoadBalancer.proto b/hbase-native-client/src/hbase/if/LoadBalancer.proto
new file mode 100644
index 0000000..f9c5d0d
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/LoadBalancer.proto
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers to represent the state of the load balancer.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "LoadBalancerProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message LoadBalancerState {
+  optional bool balancer_on = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/MapReduce.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/MapReduce.proto b/hbase-native-client/src/hbase/if/MapReduce.proto
new file mode 100644
index 0000000..f96ffdf
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/MapReduce.proto
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+
+ //This file includes protocol buffers used in MapReduce only.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MapReduceProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+message ScanMetrics {
+  repeated NameInt64Pair metrics = 1;
+}
+
+message TableSnapshotRegionSplit {
+  repeated string locations = 2;
+  optional TableSchema table = 3;
+  optional RegionInfo region = 4;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Master.proto b/hbase-native-client/src/hbase/if/Master.proto
new file mode 100644
index 0000000..ad8111e
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Master.proto
@@ -0,0 +1,828 @@
+/**
+ * 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.
+ */
+
+// All to do with the Master.  Includes schema management since these
+// changes are run by the Master process.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MasterProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "Client.proto";
+import "ClusterStatus.proto";
+import "ErrorHandling.proto";
+import "Procedure.proto";
+import "Quota.proto";
+
+/* Column-level protobufs */
+
+message AddColumnRequest {
+  required TableName table_name = 1;
+  required ColumnFamilySchema column_families = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message AddColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DeleteColumnRequest {
+  required TableName table_name = 1;
+  required bytes column_name = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message DeleteColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyColumnRequest {
+  required TableName table_name = 1;
+  required ColumnFamilySchema column_families = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message ModifyColumnResponse {
+  optional uint64 proc_id = 1;
+}
+
+/* Region-level Protos */
+
+message MoveRegionRequest {
+  required RegionSpecifier region = 1;
+  optional ServerName dest_server_name = 2;
+}
+
+message MoveRegionResponse {
+}
+
+/**
+ * Dispatch merging the specified regions.
+ */
+message DispatchMergingRegionsRequest {
+  required RegionSpecifier region_a = 1;
+  required RegionSpecifier region_b = 2;
+  optional bool forcible = 3 [default = false];
+}
+
+message DispatchMergingRegionsResponse {
+}
+
+message AssignRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message AssignRegionResponse {
+}
+
+message UnassignRegionRequest {
+  required RegionSpecifier region = 1;
+  optional bool force = 2 [default = false];
+}
+
+message UnassignRegionResponse {
+}
+
+message OfflineRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message OfflineRegionResponse {
+}
+
+/* Table-level protobufs */
+
+message CreateTableRequest {
+  required TableSchema table_schema = 1;
+  repeated bytes split_keys = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message CreateTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DeleteTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DeleteTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message TruncateTableRequest {
+  required TableName tableName = 1;
+  optional bool preserveSplits = 2 [default = false];
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message TruncateTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message EnableTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message EnableTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DisableTableRequest {
+  required TableName table_name = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DisableTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyTableRequest {
+  required TableName table_name = 1;
+  required TableSchema table_schema = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message ModifyTableResponse {
+  optional uint64 proc_id = 1;
+}
+
+/* Namespace-level protobufs */
+
+message CreateNamespaceRequest {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message CreateNamespaceResponse {
+  optional uint64 proc_id = 1;
+}
+
+message DeleteNamespaceRequest {
+  required string namespaceName = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message DeleteNamespaceResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyNamespaceRequest {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message ModifyNamespaceResponse {
+  optional uint64 proc_id = 1;
+}
+
+message GetNamespaceDescriptorRequest {
+  required string namespaceName = 1;
+}
+
+message GetNamespaceDescriptorResponse {
+  required NamespaceDescriptor namespaceDescriptor = 1;
+}
+
+message ListNamespaceDescriptorsRequest {
+}
+
+message ListNamespaceDescriptorsResponse {
+  repeated NamespaceDescriptor namespaceDescriptor = 1;
+}
+
+message ListTableDescriptorsByNamespaceRequest {
+  required string namespaceName = 1;
+}
+
+message ListTableDescriptorsByNamespaceResponse {
+  repeated TableSchema tableSchema = 1;
+}
+
+message ListTableNamesByNamespaceRequest {
+  required string namespaceName = 1;
+}
+
+message ListTableNamesByNamespaceResponse {
+  repeated TableName tableName = 1;
+}
+
+/* Cluster-level protobufs */
+
+
+message ShutdownRequest {
+}
+
+message ShutdownResponse {
+}
+
+message StopMasterRequest {
+}
+
+message StopMasterResponse {
+}
+
+message BalanceRequest {
+  optional bool force = 1;
+}
+
+message BalanceResponse {
+  required bool balancer_ran = 1;
+}
+
+message SetBalancerRunningRequest {
+  required bool on = 1;
+  optional bool synchronous = 2;
+}
+
+message SetBalancerRunningResponse {
+  optional bool prev_balance_value = 1;
+}
+
+message IsBalancerEnabledRequest {
+}
+
+message IsBalancerEnabledResponse {
+  required bool enabled = 1;
+}
+
+enum MasterSwitchType {
+  SPLIT = 0;
+  MERGE = 1;
+}
+
+message SetSplitOrMergeEnabledRequest {
+  required bool enabled = 1;
+  optional bool synchronous = 2;
+  repeated MasterSwitchType switch_types = 3;
+  optional bool skip_lock = 4;
+}
+
+message SetSplitOrMergeEnabledResponse {
+  repeated bool prev_value = 1;
+}
+
+message IsSplitOrMergeEnabledRequest {
+  required MasterSwitchType switch_type = 1;
+}
+
+message IsSplitOrMergeEnabledResponse {
+  required bool enabled = 1;
+}
+
+message ReleaseSplitOrMergeLockAndRollbackRequest {
+}
+
+message ReleaseSplitOrMergeLockAndRollbackResponse {
+}
+
+message NormalizeRequest {
+}
+
+message NormalizeResponse {
+  required bool normalizer_ran = 1;
+}
+
+message SetNormalizerRunningRequest {
+  required bool on = 1;
+}
+
+message SetNormalizerRunningResponse {
+  optional bool prev_normalizer_value = 1;
+}
+
+message IsNormalizerEnabledRequest {
+}
+
+message IsNormalizerEnabledResponse {
+  required bool enabled = 1;
+}
+
+message RunCatalogScanRequest {
+}
+
+message RunCatalogScanResponse {
+  optional int32 scan_result = 1;
+}
+
+message EnableCatalogJanitorRequest {
+  required bool enable = 1;
+}
+
+message EnableCatalogJanitorResponse {
+  optional bool prev_value = 1;
+}
+
+message IsCatalogJanitorEnabledRequest {
+}
+
+message IsCatalogJanitorEnabledResponse {
+  required bool value = 1;
+}
+
+message SnapshotRequest {
+	required SnapshotDescription snapshot = 1;
+}
+
+message SnapshotResponse {
+	required int64 expected_timeout = 1;
+}
+
+message GetCompletedSnapshotsRequest {
+}
+
+message GetCompletedSnapshotsResponse {
+	repeated SnapshotDescription snapshots = 1;
+}
+
+message DeleteSnapshotRequest {
+	required SnapshotDescription snapshot = 1;
+}
+
+message DeleteSnapshotResponse {
+}
+
+message RestoreSnapshotRequest {
+  required SnapshotDescription snapshot = 1;
+  optional uint64 nonce_group = 2 [default = 0];
+  optional uint64 nonce = 3 [default = 0];
+}
+
+message RestoreSnapshotResponse {
+  required uint64 proc_id = 1;
+}
+
+/* if you don't send the snapshot, then you will get it back
+ * in the response (if the snapshot is done) so you can check the snapshot
+ */
+message IsSnapshotDoneRequest {
+	optional SnapshotDescription snapshot = 1;
+}
+
+message IsSnapshotDoneResponse {
+	optional bool done = 1 [default = false];
+	optional SnapshotDescription snapshot = 2;
+}
+
+message IsRestoreSnapshotDoneRequest {
+  optional SnapshotDescription snapshot = 1;
+}
+
+message IsRestoreSnapshotDoneResponse {
+  optional bool done = 1 [default = false];
+}
+
+message GetSchemaAlterStatusRequest {
+  required TableName table_name = 1;
+}
+
+message GetSchemaAlterStatusResponse {
+  optional uint32 yet_to_update_regions = 1;
+  optional uint32 total_regions = 2;
+}
+
+message GetTableDescriptorsRequest {
+  repeated TableName table_names = 1;
+  optional string regex = 2;
+  optional bool include_sys_tables = 3 [default=false];
+  optional string namespace = 4;
+}
+
+message GetTableDescriptorsResponse {
+  repeated TableSchema table_schema = 1;
+}
+
+message GetTableNamesRequest {
+  optional string regex = 1;
+  optional bool include_sys_tables = 2 [default=false];
+  optional string namespace = 3;
+}
+
+message GetTableNamesResponse {
+  repeated TableName table_names = 1;
+}
+
+message GetTableStateRequest {
+  required TableName table_name = 1;
+}
+
+message GetTableStateResponse {
+  required TableState table_state = 1;
+}
+
+message GetClusterStatusRequest {
+}
+
+message GetClusterStatusResponse {
+  required ClusterStatus cluster_status = 1;
+}
+
+message IsMasterRunningRequest {
+}
+
+message IsMasterRunningResponse {
+  required bool is_master_running = 1;
+}
+
+message ExecProcedureRequest {
+  required ProcedureDescription procedure = 1;
+}
+
+message ExecProcedureResponse {
+  optional int64 expected_timeout = 1;
+  optional bytes return_data = 2;
+}
+
+message IsProcedureDoneRequest {
+  optional ProcedureDescription procedure = 1;
+}
+
+message IsProcedureDoneResponse {
+  optional bool done = 1 [default = false];
+  optional ProcedureDescription snapshot = 2;
+}
+
+message GetProcedureResultRequest {
+  required uint64 proc_id = 1;
+}
+
+message GetProcedureResultResponse {
+  enum State {
+    NOT_FOUND = 0;
+    RUNNING = 1;
+    FINISHED = 2;
+  }
+
+  required State state = 1;
+  optional uint64 start_time = 2;
+  optional uint64 last_update = 3;
+  optional bytes result = 4;
+  optional ForeignExceptionMessage exception = 5;
+}
+
+message AbortProcedureRequest {
+  required uint64 proc_id = 1;
+  optional bool mayInterruptIfRunning = 2 [default = true];
+}
+
+message AbortProcedureResponse {
+  required bool is_procedure_aborted = 1;
+}
+
+message ListProceduresRequest {
+}
+
+message ListProceduresResponse {
+  repeated Procedure procedure = 1;
+}
+
+message SetQuotaRequest {
+  optional string user_name = 1;
+  optional string user_group = 2;
+  optional string namespace = 3;
+  optional TableName table_name = 4;
+
+  optional bool remove_all = 5;
+  optional bool bypass_globals = 6;
+  optional ThrottleRequest throttle = 7;
+}
+
+message SetQuotaResponse {
+}
+
+message MajorCompactionTimestampRequest {
+  required TableName table_name = 1;
+}
+
+message MajorCompactionTimestampForRegionRequest {
+  required RegionSpecifier region = 1;
+}
+
+message MajorCompactionTimestampResponse {
+  required int64 compaction_timestamp = 1;
+}
+
+message SecurityCapabilitiesRequest {
+}
+
+message SecurityCapabilitiesResponse {
+  enum Capability {
+    SIMPLE_AUTHENTICATION = 0;
+    SECURE_AUTHENTICATION = 1;
+    AUTHORIZATION = 2;
+    CELL_AUTHORIZATION = 3;
+    CELL_VISIBILITY = 4;
+  }
+
+  repeated Capability capabilities = 1;
+}
+
+service MasterService {
+  /** Used by the client to get the number of regions that have received the updated schema */
+  rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
+    returns(GetSchemaAlterStatusResponse);
+
+  /** Get list of TableDescriptors for requested tables. */
+  rpc GetTableDescriptors(GetTableDescriptorsRequest)
+    returns(GetTableDescriptorsResponse);
+
+  /** Get the list of table names. */
+  rpc GetTableNames(GetTableNamesRequest)
+    returns(GetTableNamesResponse);
+
+  /** Return cluster status. */
+  rpc GetClusterStatus(GetClusterStatusRequest)
+    returns(GetClusterStatusResponse);
+
+  /** return true if master is available */
+  rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse);
+
+  /** Adds a column to the specified table. */
+  rpc AddColumn(AddColumnRequest)
+    returns(AddColumnResponse);
+
+  /** Deletes a column from the specified table. Table must be disabled. */
+  rpc DeleteColumn(DeleteColumnRequest)
+    returns(DeleteColumnResponse);
+
+  /** Modifies an existing column on the specified table. */
+  rpc ModifyColumn(ModifyColumnRequest)
+    returns(ModifyColumnResponse);
+
+  /** Move the region region to the destination server. */
+  rpc MoveRegion(MoveRegionRequest)
+    returns(MoveRegionResponse);
+
+ /** Master dispatch merging the regions */
+  rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
+    returns(DispatchMergingRegionsResponse);
+
+  /** Assign a region to a server chosen at random. */
+  rpc AssignRegion(AssignRegionRequest)
+    returns(AssignRegionResponse);
+
+  /**
+   * Unassign a region from current hosting regionserver.  Region will then be
+   * assigned to a regionserver chosen at random.  Region could be reassigned
+   * back to the same server.  Use MoveRegion if you want
+   * to control the region movement.
+   */
+  rpc UnassignRegion(UnassignRegionRequest)
+    returns(UnassignRegionResponse);
+
+  /**
+   * Offline a region from the assignment manager's in-memory state.  The
+   * region should be in a closed state and there will be no attempt to
+   * automatically reassign the region as in unassign.   This is a special
+   * method, and should only be used by experts or hbck.
+   */
+  rpc OfflineRegion(OfflineRegionRequest)
+    returns(OfflineRegionResponse);
+
+  /** Deletes a table */
+  rpc DeleteTable(DeleteTableRequest)
+    returns(DeleteTableResponse);
+
+  /** Truncate a table */
+  rpc truncateTable(TruncateTableRequest)
+    returns(TruncateTableResponse);
+
+  /** Puts the table on-line (only needed if table has been previously taken offline) */
+  rpc EnableTable(EnableTableRequest)
+    returns(EnableTableResponse);
+
+  /** Take table offline */
+  rpc DisableTable(DisableTableRequest)
+    returns(DisableTableResponse);
+
+  /** Modify a table's metadata */
+  rpc ModifyTable(ModifyTableRequest)
+    returns(ModifyTableResponse);
+
+  /** Creates a new table asynchronously */
+  rpc CreateTable(CreateTableRequest)
+    returns(CreateTableResponse);
+
+    /** Shutdown an HBase cluster. */
+  rpc Shutdown(ShutdownRequest)
+    returns(ShutdownResponse);
+
+  /** Stop HBase Master only.  Does not shutdown the cluster. */
+  rpc StopMaster(StopMasterRequest)
+    returns(StopMasterResponse);
+
+  /**
+   * Run the balancer.  Will run the balancer and if regions to move, it will
+   * go ahead and do the reassignments.  Can NOT run for various reasons.
+   * Check logs.
+   */
+  rpc Balance(BalanceRequest)
+    returns(BalanceResponse);
+
+  /**
+   * Turn the load balancer on or off.
+   * If synchronous is true, it waits until current balance() call, if outstanding, to return.
+   */
+  rpc SetBalancerRunning(SetBalancerRunningRequest)
+    returns(SetBalancerRunningResponse);
+
+  /**
+   * Query whether the Region Balancer is running.
+   */
+  rpc IsBalancerEnabled(IsBalancerEnabledRequest)
+    returns(IsBalancerEnabledResponse);
+
+  /**
+   * Turn the split or merge switch on or off.
+   * If synchronous is true, it waits until current operation call, if outstanding, to return.
+   */
+  rpc SetSplitOrMergeEnabled(SetSplitOrMergeEnabledRequest)
+    returns(SetSplitOrMergeEnabledResponse);
+
+  /**
+   * Query whether the split or merge switch is on/off.
+   */
+  rpc IsSplitOrMergeEnabled(IsSplitOrMergeEnabledRequest)
+    returns(IsSplitOrMergeEnabledResponse);
+
+  /**
+   * Release lock and rollback state.
+   */
+  rpc ReleaseSplitOrMergeLockAndRollback(ReleaseSplitOrMergeLockAndRollbackRequest)
+    returns(ReleaseSplitOrMergeLockAndRollbackResponse);
+
+  /**
+   * Run region normalizer. Can NOT run for various reasons. Check logs.
+   */
+  rpc Normalize(NormalizeRequest)
+    returns(NormalizeResponse);
+
+  /**
+   * Turn region normalizer on or off.
+   */
+  rpc SetNormalizerRunning(SetNormalizerRunningRequest)
+    returns(SetNormalizerRunningResponse);
+
+  /**
+   * Query whether region normalizer is enabled.
+   */
+  rpc IsNormalizerEnabled(IsNormalizerEnabledRequest)
+    returns(IsNormalizerEnabledResponse);
+
+  /** Get a run of the catalog janitor */
+  rpc RunCatalogScan(RunCatalogScanRequest)
+     returns(RunCatalogScanResponse);
+
+  /**
+   * Enable the catalog janitor on or off.
+   */
+  rpc EnableCatalogJanitor(EnableCatalogJanitorRequest)
+     returns(EnableCatalogJanitorResponse);
+
+  /**
+   * Query whether the catalog janitor is enabled.
+   */
+  rpc IsCatalogJanitorEnabled(IsCatalogJanitorEnabledRequest)
+     returns(IsCatalogJanitorEnabledResponse);
+
+  /**
+   * Call a master coprocessor endpoint
+   */
+  rpc ExecMasterService(CoprocessorServiceRequest)
+    returns(CoprocessorServiceResponse);
+
+  /**
+   * Create a snapshot for the given table.
+   */
+  rpc Snapshot(SnapshotRequest) returns(SnapshotResponse);
+
+  /**
+   * Get completed snapshots.
+   * Returns a list of snapshot descriptors for completed snapshots
+   */
+  rpc GetCompletedSnapshots(GetCompletedSnapshotsRequest) returns(GetCompletedSnapshotsResponse);
+
+  /**
+   * Delete an existing snapshot. This method can also be used to clean up an aborted snapshot.
+   */
+  rpc DeleteSnapshot(DeleteSnapshotRequest) returns(DeleteSnapshotResponse);
+
+  /**
+   * Determine if the snapshot is done yet.
+   */
+  rpc IsSnapshotDone(IsSnapshotDoneRequest) returns(IsSnapshotDoneResponse);
+
+  /**
+   * Restore a snapshot
+   */
+  rpc RestoreSnapshot(RestoreSnapshotRequest) returns(RestoreSnapshotResponse);
+
+  /**
+   * Execute a distributed procedure.
+   */
+  rpc ExecProcedure(ExecProcedureRequest) returns(ExecProcedureResponse);
+
+  /**
+   * Execute a distributed procedure with return data.
+   */
+  rpc ExecProcedureWithRet(ExecProcedureRequest) returns(ExecProcedureResponse);
+
+  /**
+   * Determine if the procedure is done yet.
+   */
+  rpc IsProcedureDone(IsProcedureDoneRequest) returns(IsProcedureDoneResponse);
+
+  /** return true if master is available */
+  /** rpc IsMasterRunning(IsMasterRunningRequest) returns(IsMasterRunningResponse); */
+
+  /** Modify a namespace's metadata */
+  rpc ModifyNamespace(ModifyNamespaceRequest)
+    returns(ModifyNamespaceResponse);
+
+  /** Creates a new namespace synchronously */
+  rpc CreateNamespace(CreateNamespaceRequest)
+    returns(CreateNamespaceResponse);
+
+  /** Deletes namespace synchronously */
+  rpc DeleteNamespace(DeleteNamespaceRequest)
+    returns(DeleteNamespaceResponse);
+
+  /** Get a namespace descriptor by name */
+  rpc GetNamespaceDescriptor(GetNamespaceDescriptorRequest)
+    returns(GetNamespaceDescriptorResponse);
+
+  /** returns a list of namespaces */
+  rpc ListNamespaceDescriptors(ListNamespaceDescriptorsRequest)
+    returns(ListNamespaceDescriptorsResponse);
+
+  /** returns a list of tables for a given namespace*/
+  rpc ListTableDescriptorsByNamespace(ListTableDescriptorsByNamespaceRequest)
+    returns(ListTableDescriptorsByNamespaceResponse);
+
+  /** returns a list of tables for a given namespace*/
+  rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
+    returns(ListTableNamesByNamespaceResponse);
+
+  /** returns table state */
+  rpc GetTableState(GetTableStateRequest)
+    returns(GetTableStateResponse);
+
+  /** Apply the new quota settings */
+  rpc SetQuota(SetQuotaRequest) returns(SetQuotaResponse);
+
+  /** Returns the timestamp of the last major compaction */
+  rpc getLastMajorCompactionTimestamp(MajorCompactionTimestampRequest)
+    returns(MajorCompactionTimestampResponse);
+
+  /** Returns the timestamp of the last major compaction */
+  rpc getLastMajorCompactionTimestampForRegion(MajorCompactionTimestampForRegionRequest)
+    returns(MajorCompactionTimestampResponse);
+
+  rpc getProcedureResult(GetProcedureResultRequest)
+    returns(GetProcedureResultResponse);
+
+  /** Returns the security capabilities in effect on the cluster */
+  rpc getSecurityCapabilities(SecurityCapabilitiesRequest)
+    returns(SecurityCapabilitiesResponse);
+
+  /** Abort a procedure */
+  rpc AbortProcedure(AbortProcedureRequest)
+    returns(AbortProcedureResponse);
+
+  /** returns a list of procedures */
+  rpc ListProcedures(ListProceduresRequest)
+    returns(ListProceduresResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/MasterProcedure.proto b/hbase-native-client/src/hbase/if/MasterProcedure.proto
new file mode 100644
index 0000000..87aae6a
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/MasterProcedure.proto
@@ -0,0 +1,285 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MasterProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "RPC.proto";
+
+// ============================================================================
+//  WARNING - Compatibility rules
+// ============================================================================
+// This .proto contains the data serialized by the master procedures.
+// Each procedure has some state stored to know, which step were executed
+// and what were the parameters or data created by the previous steps.
+// new code should be able to handle the old format or at least fail cleanly
+// triggering a rollback/cleanup.
+//
+// Procedures that are inheriting from a StateMachineProcedure have an enum:
+//  - Do not change the number of the 'State' enums.
+//    doing so, will cause executing the wrong 'step' on the pending
+//    procedures when they will be replayed.
+//  - Do not remove items from the enum, new code must be able to handle
+//    all the previous 'steps'. There may be pending procedure ready to be
+//    recovered replayed. alternative you can make sure that not-known state
+//    will result in a failure that will rollback the already executed steps.
+// ============================================================================
+
+enum CreateTableState {
+  CREATE_TABLE_PRE_OPERATION = 1;
+  CREATE_TABLE_WRITE_FS_LAYOUT = 2;
+  CREATE_TABLE_ADD_TO_META = 3;
+  CREATE_TABLE_ASSIGN_REGIONS = 4;
+  CREATE_TABLE_UPDATE_DESC_CACHE = 5;
+  CREATE_TABLE_POST_OPERATION = 6;
+}
+
+message CreateTableStateData {
+  required UserInformation user_info = 1;
+  required TableSchema table_schema = 2;
+  repeated RegionInfo region_info = 3;
+}
+
+enum ModifyTableState {
+  MODIFY_TABLE_PREPARE = 1;
+  MODIFY_TABLE_PRE_OPERATION = 2;
+  MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;
+  MODIFY_TABLE_DELETE_FS_LAYOUT = 5;
+  MODIFY_TABLE_POST_OPERATION = 6;
+  MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;
+}
+
+message ModifyTableStateData {
+  required UserInformation user_info = 1;
+  optional TableSchema unmodified_table_schema = 2;
+  required TableSchema modified_table_schema = 3;
+  required bool delete_column_family_in_modify = 4;
+}
+
+enum TruncateTableState {
+  TRUNCATE_TABLE_PRE_OPERATION = 1;
+  TRUNCATE_TABLE_REMOVE_FROM_META = 2;
+  TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;
+  TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;
+  TRUNCATE_TABLE_ADD_TO_META = 5;
+  TRUNCATE_TABLE_ASSIGN_REGIONS = 6;
+  TRUNCATE_TABLE_POST_OPERATION = 7;
+}
+
+message TruncateTableStateData {
+  required UserInformation user_info = 1;
+  required bool preserve_splits = 2;
+  optional TableName table_name = 3;
+  optional TableSchema table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}
+
+enum DeleteTableState {
+  DELETE_TABLE_PRE_OPERATION = 1;
+  DELETE_TABLE_REMOVE_FROM_META = 2;
+  DELETE_TABLE_CLEAR_FS_LAYOUT = 3;
+  DELETE_TABLE_UPDATE_DESC_CACHE = 4;
+  DELETE_TABLE_UNASSIGN_REGIONS = 5;
+  DELETE_TABLE_POST_OPERATION = 6;
+}
+
+message DeleteTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  repeated RegionInfo region_info = 3;
+}
+
+enum CreateNamespaceState {
+  CREATE_NAMESPACE_PREPARE = 1;
+  CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
+  CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
+  CREATE_NAMESPACE_UPDATE_ZK = 4;
+  CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
+}
+
+message CreateNamespaceStateData {
+  required NamespaceDescriptor namespace_descriptor = 1;
+}
+
+enum ModifyNamespaceState {
+  MODIFY_NAMESPACE_PREPARE = 1;
+  MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
+  MODIFY_NAMESPACE_UPDATE_ZK = 3;
+}
+
+message ModifyNamespaceStateData {
+  required NamespaceDescriptor namespace_descriptor = 1;
+  optional NamespaceDescriptor unmodified_namespace_descriptor = 2;
+}
+
+enum DeleteNamespaceState {
+  DELETE_NAMESPACE_PREPARE = 1;
+  DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
+  DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
+  DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
+  DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
+}
+
+message DeleteNamespaceStateData {
+  required string namespace_name = 1;
+  optional NamespaceDescriptor namespace_descriptor = 2;
+}
+
+enum AddColumnFamilyState {
+  ADD_COLUMN_FAMILY_PREPARE = 1;
+  ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
+  ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  ADD_COLUMN_FAMILY_POST_OPERATION = 4;
+  ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message AddColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum ModifyColumnFamilyState {
+  MODIFY_COLUMN_FAMILY_PREPARE = 1;
+  MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
+  MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
+  MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message ModifyColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum DeleteColumnFamilyState {
+  DELETE_COLUMN_FAMILY_PREPARE = 1;
+  DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
+  DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
+  DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
+  DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
+}
+
+message DeleteColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bytes columnfamily_name = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum EnableTableState {
+  ENABLE_TABLE_PREPARE = 1;
+  ENABLE_TABLE_PRE_OPERATION = 2;
+  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
+  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
+  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
+  ENABLE_TABLE_POST_OPERATION = 6;
+}
+
+message EnableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+enum DisableTableState {
+  DISABLE_TABLE_PREPARE = 1;
+  DISABLE_TABLE_PRE_OPERATION = 2;
+  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
+  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
+  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
+  DISABLE_TABLE_POST_OPERATION = 6;
+}
+
+message DisableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+message RestoreParentToChildRegionsPair {
+  required string parent_region_name = 1;
+  required string child1_region_name = 2;
+  required string child2_region_name = 3;
+}
+
+enum CloneSnapshotState {
+  CLONE_SNAPSHOT_PRE_OPERATION = 1;
+  CLONE_SNAPSHOT_WRITE_FS_LAYOUT = 2;
+  CLONE_SNAPSHOT_ADD_TO_META = 3;
+  CLONE_SNAPSHOT_ASSIGN_REGIONS = 4;
+  CLONE_SNAPSHOT_UPDATE_DESC_CACHE = 5;
+  CLONE_SNAPSHOT_POST_OPERATION = 6;
+}
+
+message CloneSnapshotStateData {
+  required UserInformation user_info = 1;
+  required SnapshotDescription snapshot = 2;
+  required TableSchema table_schema = 3;
+  repeated RegionInfo region_info = 4;
+  repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5;
+}
+
+enum RestoreSnapshotState {
+  RESTORE_SNAPSHOT_PRE_OPERATION = 1;
+  RESTORE_SNAPSHOT_UPDATE_TABLE_DESCRIPTOR = 2;
+  RESTORE_SNAPSHOT_WRITE_FS_LAYOUT = 3;
+  RESTORE_SNAPSHOT_UPDATE_META = 4;
+}
+
+message RestoreSnapshotStateData {
+  required UserInformation user_info = 1;
+  required SnapshotDescription snapshot = 2;
+  required TableSchema modified_table_schema = 3;
+  repeated RegionInfo region_info_for_restore = 4;
+  repeated RegionInfo region_info_for_remove = 5;
+  repeated RegionInfo region_info_for_add = 6;
+  repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
+}
+
+message ServerCrashStateData {
+  required ServerName server_name = 1;
+  optional bool distributed_log_replay = 2;
+  repeated RegionInfo regions_on_crashed_server = 3;
+  repeated RegionInfo regions_assigned = 4;
+  optional bool carrying_meta = 5;
+  optional bool should_split_wal = 6 [default = true];
+}
+
+enum ServerCrashState {
+  SERVER_CRASH_START = 1;
+  SERVER_CRASH_PROCESS_META = 2;
+  SERVER_CRASH_GET_REGIONS = 3;
+  SERVER_CRASH_NO_SPLIT_LOGS = 4;
+  SERVER_CRASH_SPLIT_LOGS = 5;
+  SERVER_CRASH_PREPARE_LOG_REPLAY = 6;
+  // Removed SERVER_CRASH_CALC_REGIONS_TO_ASSIGN = 7;
+  SERVER_CRASH_ASSIGN = 8;
+  SERVER_CRASH_WAIT_ON_ASSIGN = 9;
+  SERVER_CRASH_FINISH = 100;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/MultiRowMutation.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/MultiRowMutation.proto b/hbase-native-client/src/hbase/if/MultiRowMutation.proto
new file mode 100644
index 0000000..747afac
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/MultiRowMutation.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+import "Client.proto";
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MultiRowMutationProtos";
+option java_generate_equals_and_hash = true;
+option java_generic_services = true;
+option optimize_for = SPEED;
+
+message MultiRowMutationProcessorRequest{
+}
+
+message MultiRowMutationProcessorResponse{
+}
+
+message MutateRowsRequest {
+  repeated MutationProto mutation_request = 1;
+  optional uint64 nonce_group = 2;
+  optional uint64 nonce = 3;
+}
+
+message MutateRowsResponse {
+}
+
+service MultiRowMutationService {
+  rpc MutateRows(MutateRowsRequest)
+      returns(MutateRowsResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Procedure.proto b/hbase-native-client/src/hbase/if/Procedure.proto
new file mode 100644
index 0000000..55e44a4
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Procedure.proto
@@ -0,0 +1,119 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "ErrorHandling.proto";
+
+enum ProcedureState {
+  INITIALIZING = 1;         // Procedure in construction, not yet added to the executor
+  RUNNABLE = 2;             // Procedure added to the executor, and ready to be executed
+  WAITING = 3;              // The procedure is waiting on children to be completed
+  WAITING_TIMEOUT = 4;      // The procedure is waiting a timout or an external event
+  ROLLEDBACK = 5;           // The procedure failed and was rolledback
+  FINISHED = 6;             // The procedure execution is completed. may need a rollback if failed.
+}
+
+/**
+ * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
+ */
+message Procedure {
+  // internal "static" state
+  required string class_name = 1;        // full classname to be able to instantiate the procedure
+  optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
+  required uint64 proc_id = 3;
+  required uint64 start_time = 4;
+  optional string owner = 5;
+
+  // internal "runtime" state
+  required ProcedureState state = 6;
+  repeated uint32 stack_id = 7;          // stack indices in case the procedure was running
+  required uint64 last_update = 8;
+  optional uint32 timeout = 9;
+
+  // user state/results
+  optional ForeignExceptionMessage exception = 10;
+  optional bytes result = 11;           // opaque (user) result structure
+  optional bytes state_data = 12;       // opaque (user) procedure internal-state
+
+  // Nonce to prevent same procedure submit by multiple times
+  optional uint64 nonce_group = 13 [default = 0];
+  optional uint64 nonce = 14 [default = 0];
+}
+
+/**
+ * SequentialProcedure data
+ */
+message SequentialProcedureData {
+  required bool executed = 1;
+}
+
+/**
+ * StateMachineProcedure data
+ */
+message StateMachineProcedureData {
+  repeated uint32 state = 1;
+}
+
+/**
+ * Procedure WAL header
+ */
+message ProcedureWALHeader {
+  required uint32 version = 1;
+  required uint32 type = 2;
+  required uint64 log_id = 3;
+  required uint64 min_proc_id = 4;
+}
+
+/**
+ * Procedure WAL trailer
+ */
+message ProcedureWALTrailer {
+  required uint32 version = 1;
+  required uint64 tracker_pos = 2;
+}
+
+message ProcedureStoreTracker {
+  message TrackerNode {
+    required uint64 start_id = 1;
+    repeated uint64 updated = 2;
+    repeated uint64 deleted = 3;
+  }
+
+  repeated TrackerNode node = 1;
+}
+
+message ProcedureWALEntry {
+  enum Type {
+    PROCEDURE_WAL_EOF     = 1;
+    PROCEDURE_WAL_INIT    = 2;
+    PROCEDURE_WAL_INSERT  = 3;
+    PROCEDURE_WAL_UPDATE  = 4;
+    PROCEDURE_WAL_DELETE  = 5;
+    PROCEDURE_WAL_COMPACT = 6;
+  }
+
+  required Type type = 1;
+  repeated Procedure procedure = 2;
+  optional uint64 proc_id = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/Quota.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/Quota.proto b/hbase-native-client/src/hbase/if/Quota.proto
new file mode 100644
index 0000000..a8303b1
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/Quota.proto
@@ -0,0 +1,76 @@
+ /**
+ * 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.
+ */
+
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "QuotaProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+enum QuotaScope {
+  CLUSTER = 1;
+  MACHINE = 2;
+}
+
+message TimedQuota {
+  required TimeUnit time_unit = 1;
+  optional uint64 soft_limit  = 2;
+  optional float share = 3;
+  optional QuotaScope scope  = 4 [default = MACHINE];
+}
+
+enum ThrottleType {
+  REQUEST_NUMBER = 1;
+  REQUEST_SIZE   = 2;
+  WRITE_NUMBER   = 3;
+  WRITE_SIZE     = 4;
+  READ_NUMBER    = 5;
+  READ_SIZE      = 6;
+}
+
+message Throttle {
+  optional TimedQuota req_num  = 1;
+  optional TimedQuota req_size = 2;
+
+  optional TimedQuota write_num  = 3;
+  optional TimedQuota write_size = 4;
+
+  optional TimedQuota read_num  = 5;
+  optional TimedQuota read_size = 6;
+}
+
+message ThrottleRequest {
+  optional ThrottleType type = 1;
+  optional TimedQuota timed_quota = 2;
+}
+
+enum QuotaType {
+  THROTTLE = 1;
+}
+
+message Quotas {
+  optional bool bypass_globals = 1 [default = false];
+  optional Throttle throttle = 2;
+}
+
+message QuotaUsage {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RPC.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RPC.proto b/hbase-native-client/src/hbase/if/RPC.proto
new file mode 100644
index 0000000..8413d25
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RPC.proto
@@ -0,0 +1,137 @@
+/**
+ * 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.
+ */
+package hbase.pb;
+
+import "Tracing.proto"; 
+import "HBase.proto";
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RPCProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+// See https://issues.apache.org/jira/browse/HBASE-7898 for high-level
+// description of RPC specification.
+//
+// On connection setup, the client sends six bytes of preamble -- a four
+// byte magic, a byte of version, and a byte of authentication type.
+//
+// We then send a "ConnectionHeader" protobuf of user information and the
+// 'protocol' or 'service' that is to be run over this connection as well as
+// info such as codecs and compression to use when we send cell blocks(see below).
+// This connection header protobuf is prefaced by an int that holds the length
+// of this connection header (this is NOT a varint).  The pb connection header
+// is sent with Message#writeTo.  The server throws an exception if it doesn't
+// like what it was sent noting what it is objecting too.  Otherwise, the server
+// says nothing and is open for business.
+//
+// Hereafter the client makes requests and the server returns responses.
+//
+// Requests look like this:
+//
+// <An int with the total length of the request>
+// <RequestHeader Message written out using Message#writeDelimitedTo>
+// <Optionally a Request Parameter Message written out using Message#writeDelimitedTo>
+// <Optionally a Cell block>
+//
+// ...where the Request Parameter Message is whatever the method name stipulated
+// in the RequestHeader expects; e.g. if the method is a scan, then the pb
+// Request Message is a GetRequest, or a ScanRequest.  A block of Cells
+// optionally follows.  The presence of a Request param Message and/or a
+// block of Cells will be noted in the RequestHeader.
+//
+// Response is the mirror of the request:
+//
+// <An int with the total length of the response>
+// <ResponseHeader Message written out using Message#writeDelimitedTo>
+// <Optionally a Response Result Message written out using Message#writeDelimitedTo>
+// <Optionally a Cell block>
+//
+// ...where the Response Message is the response type that goes with the
+// method specified when making the request and the follow on Cell blocks may
+// or may not be there -- read the response header to find out if one following.
+// If an exception, it will be included inside the Response Header.
+//
+// Any time we write a pb, we do it with Message#writeDelimitedTo EXCEPT when
+// the connection header is sent; this is prefaced by an int with its length
+// and the pb connection header is then written with Message#writeTo.
+//
+
+// User Information proto.  Included in ConnectionHeader on connection setup
+message UserInformation {
+  required string effective_user = 1;
+  optional string real_user = 2;
+}
+
+// This is sent on connection setup after the connection preamble is sent.
+message ConnectionHeader {
+  optional UserInformation user_info = 1;
+  optional string service_name = 2;
+  // Cell block codec we will use sending over optional cell blocks.  Server throws exception
+  // if cannot deal.  Null means no codec'ing going on so we are pb all the time (SLOW!!!)
+  optional string cell_block_codec_class = 3;
+  // Compressor we will use if cell block is compressed.  Server will throw exception if not supported.
+  // Class must implement hadoop's CompressionCodec Interface.  Can't compress if no codec.
+  optional string cell_block_compressor_class = 4;
+  optional VersionInfo version_info = 5;
+}
+
+// Optional Cell block Message.  Included in client RequestHeader
+message CellBlockMeta {
+  // Length of the following cell block.  Could calculate it but convenient having it too hand.
+  optional uint32 length = 1;
+}
+
+// At the RPC layer, this message is used to carry
+// the server side exception to the RPC client.
+message ExceptionResponse {
+  // Class name of the exception thrown from the server
+  optional string exception_class_name = 1;
+  // Exception stack trace from the server side
+  optional string stack_trace = 2;
+  // Optional hostname.  Filled in for some exceptions such as region moved
+  // where exception gives clue on where the region may have moved.
+  optional string hostname = 3;
+  optional int32 port = 4;
+  // Set if we are NOT to retry on receipt of this exception
+  optional bool do_not_retry = 5;
+}
+
+// Header sent making a request.
+message RequestHeader {
+  // Monotonically increasing call_id to keep track of RPC requests and their response
+  optional uint32 call_id = 1;
+  optional RPCTInfo trace_info = 2;
+  optional string method_name = 3;
+  // If true, then a pb Message param follows.
+  optional bool request_param = 4;
+  // If present, then an encoded data block follows.
+  optional CellBlockMeta cell_block_meta = 5;
+  // 0 is NORMAL priority.  200 is HIGH.  If no priority, treat it as NORMAL.
+  // See HConstants.
+  optional uint32 priority = 6;
+  optional uint32 timeout = 7;
+}
+
+message ResponseHeader {
+  optional uint32 call_id = 1;
+  // If present, then request threw an exception and no response message (else we presume one)
+  optional ExceptionResponse exception = 2;
+  // If present, then an encoded data block follows.
+  optional CellBlockMeta cell_block_meta = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RSGroup.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RSGroup.proto b/hbase-native-client/src/hbase/if/RSGroup.proto
new file mode 100644
index 0000000..7358941
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RSGroup.proto
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RSGroupProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+message RSGroupInfo {
+  required string name = 1;
+  repeated ServerName servers = 4;
+  repeated TableName tables = 3;
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RSGroupAdmin.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RSGroupAdmin.proto b/hbase-native-client/src/hbase/if/RSGroupAdmin.proto
new file mode 100644
index 0000000..fda9b09
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RSGroupAdmin.proto
@@ -0,0 +1,136 @@
+/**
+ * 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.
+ */
+
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RSGroupAdminProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "RSGroup.proto";
+
+/** Group level protobufs */
+
+message ListTablesOfRSGroupRequest {
+  required string r_s_group_name = 1;
+}
+
+message ListTablesOfRSGroupResponse {
+  repeated TableName table_name = 1;
+}
+
+message GetRSGroupInfoRequest {
+  required string r_s_group_name = 1;
+}
+
+message GetRSGroupInfoResponse {
+  optional RSGroupInfo r_s_group_info = 1;
+}
+
+message GetRSGroupInfoOfTableRequest {
+  required TableName table_name = 1;
+}
+
+message GetRSGroupInfoOfTableResponse {
+  optional RSGroupInfo r_s_group_info = 1;
+}
+
+message MoveServersRequest {
+  required string target_group = 1;
+  repeated ServerName servers = 3;
+}
+
+message MoveServersResponse {
+}
+
+message MoveTablesRequest {
+  required string target_group = 1;
+  repeated TableName table_name = 2;
+}
+
+message MoveTablesResponse {
+}
+
+message AddRSGroupRequest {
+  required string r_s_group_name = 1;
+}
+
+message AddRSGroupResponse {
+}
+
+message RemoveRSGroupRequest {
+  required string r_s_group_name = 1;
+}
+
+message RemoveRSGroupResponse {
+}
+
+message BalanceRSGroupRequest {
+  required string r_s_group_name = 1;
+}
+
+message BalanceRSGroupResponse {
+  required bool balanceRan = 1;
+}
+
+message ListRSGroupInfosRequest {
+}
+
+message ListRSGroupInfosResponse {
+  repeated RSGroupInfo  r_s_group_info = 1;
+}
+
+message GetRSGroupInfoOfServerRequest {
+  required ServerName server = 2;
+}
+
+message GetRSGroupInfoOfServerResponse {
+  optional RSGroupInfo r_s_group_info = 1;
+}
+
+service RSGroupAdminService {
+  rpc GetRSGroupInfo(GetRSGroupInfoRequest)
+    returns (GetRSGroupInfoResponse);
+
+  rpc GetRSGroupInfoOfTable(GetRSGroupInfoOfTableRequest)
+    returns (GetRSGroupInfoOfTableResponse);
+
+  rpc GetRSGroupInfoOfServer(GetRSGroupInfoOfServerRequest)
+    returns (GetRSGroupInfoOfServerResponse);
+
+  rpc MoveServers(MoveServersRequest)
+    returns (MoveServersResponse);
+
+  rpc MoveTables(MoveTablesRequest)
+    returns (MoveTablesResponse);
+
+  rpc AddRSGroup(AddRSGroupRequest)
+    returns (AddRSGroupResponse);
+
+  rpc RemoveRSGroup(RemoveRSGroupRequest)
+    returns (RemoveRSGroupResponse);
+
+  rpc BalanceRSGroup(BalanceRSGroupRequest)
+    returns (BalanceRSGroupResponse);
+
+  rpc ListRSGroupInfos(ListRSGroupInfosRequest)
+    returns (ListRSGroupInfosResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RegionNormalizer.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RegionNormalizer.proto b/hbase-native-client/src/hbase/if/RegionNormalizer.proto
new file mode 100644
index 0000000..e5305d6
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RegionNormalizer.proto
@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers to represent the state of the load balancer.
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RegionNormalizerProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message RegionNormalizerState {
+    optional bool normalizer_on = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RegionServerStatus.proto b/hbase-native-client/src/hbase/if/RegionServerStatus.proto
new file mode 100644
index 0000000..fda9de2
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RegionServerStatus.proto
@@ -0,0 +1,158 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used for RegionServerStatusProtocol.
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RegionServerStatusProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "ClusterStatus.proto";
+
+message RegionServerStartupRequest {
+  /** Port number this regionserver is up on */
+  required uint32 port = 1;
+
+  /** This servers' startcode */
+  required uint64 server_start_code = 2;
+
+  /** Current time of the region server in ms */
+  required uint64 server_current_time = 3;
+
+  /** hostname for region server, optional */
+  optional string use_this_hostname_instead = 4;
+}
+
+message RegionServerStartupResponse {
+  /**
+   * Configuration for the regionserver to use: e.g. filesystem,
+   * hbase rootdir, the hostname to use creating the RegionServer ServerName,
+   * etc
+   */
+  repeated NameStringPair map_entries = 1;
+}
+
+message RegionServerReportRequest {
+  required ServerName server = 1;
+
+  /** load the server is under */
+  optional ServerLoad load = 2;
+}
+
+message RegionServerReportResponse {
+}
+
+message ReportRSFatalErrorRequest {
+  /** name of the server experiencing the error */
+  required ServerName server = 1;
+
+  /** informative text to expose in the master logs and UI */
+  required string error_message = 2;
+}
+
+message ReportRSFatalErrorResponse {
+}
+
+message GetLastFlushedSequenceIdRequest {
+  /** region name */
+  required bytes region_name = 1;
+}
+
+message GetLastFlushedSequenceIdResponse {
+  /** the last WAL sequence id flushed from MemStore to HFile for the region */
+  required uint64 last_flushed_sequence_id = 1;
+
+  /** the last WAL sequence id flushed from MemStore to HFile for stores of the region */
+  repeated StoreSequenceId store_last_flushed_sequence_id = 2;
+}
+
+message RegionStateTransition {
+  required TransitionCode transition_code = 1;
+
+  /** Mutliple regions are involved during merging/splitting */
+  repeated RegionInfo region_info = 2;
+
+  /** For newly opened region, the open seq num is needed */
+  optional uint64 open_seq_num = 3;
+
+  enum TransitionCode {
+    OPENED = 0;
+    FAILED_OPEN = 1;
+    /** No failed_close, in which case region server will abort */
+    CLOSED = 2;
+
+    /** Ask master for ok to split/merge region(s) */
+    READY_TO_SPLIT = 3;
+    READY_TO_MERGE = 4;
+
+    SPLIT_PONR = 5;
+    MERGE_PONR = 6;
+
+    SPLIT = 7;
+    MERGED = 8;
+    SPLIT_REVERTED = 9;
+    MERGE_REVERTED = 10;
+  }
+}
+
+message ReportRegionStateTransitionRequest {
+  /** This region server's server name */
+  required ServerName server = 1;
+
+  repeated RegionStateTransition transition = 2;
+}
+
+message ReportRegionStateTransitionResponse {
+  /** Error message if failed to update the region state */
+  optional string error_message = 1;
+}
+
+service RegionServerStatusService {
+  /** Called when a region server first starts. */
+  rpc RegionServerStartup(RegionServerStartupRequest)
+    returns(RegionServerStartupResponse);
+
+  /** Called to report the load the RegionServer is under. */
+  rpc RegionServerReport(RegionServerReportRequest)
+    returns(RegionServerReportResponse);
+
+  /**
+   * Called by a region server to report a fatal error that is causing it to
+   * abort.
+   */
+  rpc ReportRSFatalError(ReportRSFatalErrorRequest)
+    returns(ReportRSFatalErrorResponse);
+
+  /** Called to get the sequence id of the last MemStore entry flushed to an
+   * HFile for a specified region. Used by the region server to speed up
+   * log splitting. */
+  rpc GetLastFlushedSequenceId(GetLastFlushedSequenceIdRequest)
+    returns(GetLastFlushedSequenceIdResponse);
+
+  /**
+   * Called by a region server to report the progress of a region
+   * transition. If the request fails, the transition should
+   * be aborted.
+   */
+  rpc ReportRegionStateTransition(ReportRegionStateTransitionRequest)
+    returns(ReportRegionStateTransitionResponse);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/src/hbase/if/RowProcessor.proto
----------------------------------------------------------------------
diff --git a/hbase-native-client/src/hbase/if/RowProcessor.proto b/hbase-native-client/src/hbase/if/RowProcessor.proto
new file mode 100644
index 0000000..cf2f30f
--- /dev/null
+++ b/hbase-native-client/src/hbase/if/RowProcessor.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+/**
+ * Defines a protocol to perform multi row transactions.
+ * See BaseRowProcessorEndpoint for the implementation.
+ * See HRegion#processRowsWithLocks() for details.
+ */
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RowProcessorProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+message ProcessRequest {
+  required string row_processor_class_name = 1;
+  optional string row_processor_initializer_message_name = 2;
+  optional bytes  row_processor_initializer_message = 3;
+  optional uint64 nonce_group = 4;
+  optional uint64 nonce = 5;
+}
+
+message ProcessResponse {
+  required bytes row_processor_result = 1;
+}
+
+service RowProcessorService {
+  rpc Process(ProcessRequest) returns (ProcessResponse);
+}


[21/25] hbase git commit: HBASE-18725 [C++] Install header files as well as library

Posted by en...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/cell.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.cc b/hbase-native-client/core/cell.cc
deleted file mode 100644
index e475d49..0000000
--- a/hbase-native-client/core/cell.cc
+++ /dev/null
@@ -1,123 +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/cell.h"
-#include <climits>
-#include <limits>
-#include <stdexcept>
-
-#include "folly/Conv.h"
-#include "utils/bytes-util.h"
-
-namespace hbase {
-
-Cell::Cell(const std::string &row, const std::string &family, const std::string &qualifier,
-           const int64_t timestamp, const std::string &value, const hbase::CellType &cell_type)
-    : row_(row),
-      family_(family),
-      qualifier_(qualifier),
-      timestamp_(timestamp),
-      cell_type_(cell_type),
-      value_(value),
-      sequence_id_(0) {
-  if (0 == row.size()) throw std::runtime_error("Row size should be greater than 0");
-
-  if (0 >= timestamp) throw std::runtime_error("Timestamp should be greater than 0");
-}
-
-Cell::Cell(const Cell &cell)
-    : row_(cell.row_),
-      family_(cell.family_),
-      qualifier_(cell.qualifier_),
-      timestamp_(cell.timestamp_),
-      cell_type_(cell.cell_type_),
-      value_(cell.value_),
-      sequence_id_(cell.sequence_id_) {}
-
-Cell &Cell::operator=(const Cell &cell) {
-  row_ = cell.row_;
-  family_ = cell.family_;
-  qualifier_ = cell.qualifier_;
-  timestamp_ = cell.timestamp_;
-  cell_type_ = cell.cell_type_;
-  value_ = cell.value_;
-  sequence_id_ = cell.sequence_id_;
-
-  return *this;
-}
-
-Cell::~Cell() {}
-
-const std::string &Cell::Row() const { return row_; }
-
-const std::string &Cell::Family() const { return family_; }
-
-const std::string &Cell::Qualifier() const { return qualifier_; }
-
-int64_t Cell::Timestamp() const { return timestamp_; }
-
-const std::string &Cell::Value() const { return value_; }
-
-hbase::CellType Cell::Type() const { return cell_type_; }
-
-int64_t Cell::SequenceId() const { return sequence_id_; }
-
-std::string Cell::DebugString() const {
-  std::string timestamp_str;
-  if (timestamp_ == std::numeric_limits<int64_t>::max()) {
-    timestamp_str = "LATEST_TIMESTAMP";
-  } else {
-    timestamp_str = folly::to<std::string>(timestamp_);
-  }
-
-  return BytesUtil::ToStringBinary(row_) + "/" + BytesUtil::ToStringBinary(family_) +
-         (family_.empty() ? "" : ":") + BytesUtil::ToStringBinary(qualifier_) + "/" +
-         timestamp_str + "/" + TypeToString(cell_type_) + "/vlen=" +
-         folly::to<std::string>(value_.size()) + "/seqid=" + folly::to<std::string>(sequence_id_);
-}
-
-const char *Cell::TypeToString(CellType type) {
-  switch (type) {
-    case CellType::MINIMUM:
-      return "MINIMUM";
-    case CellType::PUT:
-      return "PUT";
-    case CellType::DELETE:
-      return "DELETE";
-    case CellType::DELETE_COLUMN:
-      return "DELETE_COLUMN";
-    case CellType::DELETE_FAMILY:
-      return "DELETE_FAMILY";
-    case CellType::MAXIMUM:
-      return "MAXIMUM";
-    default:
-      return "UNKNOWN";
-  }
-}
-
-size_t Cell::EstimatedSize() const {
-  size_t s = sizeof(Cell);
-  s += row_.capacity();
-  s += family_.capacity();
-  s += qualifier_.capacity();
-  s += value_.capacity();
-  return s;
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/cell.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.h b/hbase-native-client/core/cell.h
deleted file mode 100644
index 7a62a9b..0000000
--- a/hbase-native-client/core/cell.h
+++ /dev/null
@@ -1,72 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <string>
-
-namespace hbase {
-
-enum class CellType {
-  MINIMUM = 0,
-  PUT = 4,
-  DELETE = 8,
-  DELETE_FAMILY_VERSION = 10,
-  DELETE_COLUMN = 12,
-  DELETE_FAMILY = 14,
-  MAXIMUM = 255
-};
-
-class Cell {
- public:
-  Cell(const std::string &row, const std::string &family, const std::string &qualifier,
-       const int64_t timestamp, const std::string &value, const hbase::CellType &cell_type);
-  Cell(const Cell &cell);
-  Cell &operator=(const Cell &cell);
-  virtual ~Cell();
-  const std::string &Row() const;
-  const std::string &Family() const;
-  const std::string &Qualifier() const;
-  int64_t Timestamp() const;
-  const std::string &Value() const;
-  CellType Type() const;
-  int64_t SequenceId() const;
-  std::string DebugString() const;
-  /** Returns estimated size of the Cell object including deep heap space usage
-    * of its data. Notice that this is a very rough estimate. */
-  size_t EstimatedSize() const;
-
- private:
-  std::string row_;
-  std::string family_;
-  std::string qualifier_;
-  // Since java does not have unsigned, we are also using signed numerics here
-  // so that we won't have surprises when large uint64's are treated as
-  // negative values in the java server side
-  int64_t timestamp_;
-  hbase::CellType cell_type_;
-  std::string value_;
-  int64_t sequence_id_;
-
- private:
-  static const char *TypeToString(CellType type);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/client-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/client-test.cc b/hbase-native-client/core/client-test.cc
deleted file mode 100644
index 3f72880..0000000
--- a/hbase-native-client/core/client-test.cc
+++ /dev/null
@@ -1,697 +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 <gtest/gtest.h>
-
-#include "core/append.h"
-#include "core/cell.h"
-#include "core/client.h"
-#include "core/configuration.h"
-#include "core/delete.h"
-#include "core/get.h"
-#include "core/hbase-configuration-loader.h"
-#include "core/increment.h"
-#include "core/put.h"
-#include "core/result.h"
-#include "core/table.h"
-#include "exceptions/exception.h"
-#include "serde/table-name.h"
-#include "test-util/test-util.h"
-#include "utils/bytes-util.h"
-
-using hbase::Cell;
-using hbase::Configuration;
-using hbase::Get;
-using hbase::RetriesExhaustedException;
-using hbase::Put;
-using hbase::Table;
-using hbase::TestUtil;
-using std::experimental::nullopt;
-
-class ClientTest : public ::testing::Test {
- public:
-  static const constexpr char *kDefHBaseConfPath = "./build/test-data/client-test/conf/";
-  static const constexpr char *kHBaseDefaultXml = "hbase-default.xml";
-  static const constexpr char *kHBaseSiteXml = "hbase-site.xml";
-  static const constexpr char *kHBaseXmlData =
-      "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
-      "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
-      "Software Foundation (ASF) under one\n * or more contributor license "
-      "agreements.  See the NOTICE file\n * distributed with this work for "
-      "additional information\n * regarding copyright ownership.  The ASF "
-      "licenses this file\n * to you under the Apache License, Version 2.0 "
-      "(the\n * \"License\"); you may not use this file except in compliance\n * "
-      "with the License.  You may obtain a copy of the License at\n *\n *     "
-      "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
-      "applicable law or agreed to in writing, software\n * distributed under "
-      "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
-      "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
-      "for the specific language governing permissions and\n * limitations under "
-      "the License.\n "
-      "*/\n-->\n<configuration>\n\n</configuration>";
-
-  static void WriteDataToFile(const std::string &file, const std::string &xml_data) {
-    std::ofstream hbase_conf;
-    hbase_conf.open(file.c_str());
-    hbase_conf << xml_data;
-    hbase_conf.close();
-  }
-
-  static void CreateHBaseConf(const std::string &dir, const std::string &file,
-                              const std::string xml_data) {
-    // Remove temp file always
-    boost::filesystem::remove((dir + file).c_str());
-    boost::filesystem::create_directories(dir.c_str());
-    WriteDataToFile((dir + file), xml_data);
-  }
-
-  static void CreateHBaseConfWithEnv() {
-    // Creating Empty Config Files so that we dont get a Configuration exception @Client
-    CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseXmlData);
-    // the hbase-site.xml would be persisted by MiniCluster
-    setenv("HBASE_CONF", kDefHBaseConfPath, 1);
-  }
-  static std::unique_ptr<hbase::TestUtil> test_util;
-
-  static void SetUpTestCase() {
-    google::InstallFailureSignalHandler();
-    test_util = std::make_unique<hbase::TestUtil>();
-    test_util->StartMiniCluster(2);
-  }
-};
-std::unique_ptr<hbase::TestUtil> ClientTest::test_util = nullptr;
-
-TEST_F(ClientTest, EmptyConfigurationPassedToClient) { ASSERT_ANY_THROW(hbase::Client client); }
-
-TEST_F(ClientTest, ConfigurationPassedToClient) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  ClientTest::CreateHBaseConfWithEnv();
-
-  // Create Configuration
-  hbase::HBaseConfigurationLoader loader;
-  auto conf = loader.LoadDefaultResources();
-  // Create a client
-  hbase::Client client(conf.value());
-  client.Close();
-}
-
-TEST_F(ClientTest, DefaultConfiguration) {
-  // Remove already configured env if present.
-  unsetenv("HBASE_CONF");
-  ClientTest::CreateHBaseConfWithEnv();
-
-  // Create Configuration
-  hbase::Client client;
-  client.Close();
-}
-
-TEST_F(ClientTest, Append) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("t", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-  std::string val1 = "a";
-  auto result = table->Append(hbase::Append{row}.Add("d", "1", val1));
-
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ(row, result->Row());
-  EXPECT_EQ(val1, *(result->Value("d", "1")));
-
-  std::string val2 = "b";
-  result = table->Append(hbase::Append{row}.Add("d", "1", val2));
-
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ(row, result->Row());
-  EXPECT_EQ("ab", *(result->Value("d", "1")));
-}
-
-TEST_F(ClientTest, PutGetDelete) {
-  // Using TestUtil to populate test data
-  std::string tableName = "t1";
-  ClientTest::test_util->CreateTable(tableName, "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(tableName);
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  // Perform Puts
-  std::string valExtra = "value for extra";
-  std::string valExt = "value for ext";
-  table->Put(Put{row}.AddColumn("d", "1", "value1"));
-  // Put two values for column "extra"
-  table->Put(Put{row}.AddColumn("d", "extra", "1st val extra"));
-  usleep(1000);
-  table->Put(Put{row}.AddColumn("d", "extra", valExtra));
-  table->Put(Put{row}.AddColumn("d", "ext", valExt));
-
-  // Perform the Get
-  hbase::Get get(row);
-  auto result = table->Get(get);
-
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ("test1", result->Row());
-  EXPECT_EQ("value1", *(result->Value("d", "1")));
-  EXPECT_EQ(valExtra, *(result->Value("d", "extra")));
-  auto cell = *(result->ColumnCells("d", "extra"))[0];
-  auto tsExtra = cell.Timestamp();
-  auto tsExt = (*(result->ColumnCells("d", "ext"))[0]).Timestamp();
-
-  // delete column "1"
-  table->Delete(hbase::Delete{row}.AddColumn("d", "1"));
-  result = table->Get(get);
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
-  EXPECT_EQ(valExtra, *(result->Value("d", "extra")));
-
-  // delete cell from column "extra" with timestamp tsExtra
-  table->Delete(hbase::Delete{row}.AddColumn("d", "extra", tsExtra));
-  result = table->Get(get);
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
-  ASSERT_TRUE(result->Value("d", "extra") != nullopt) << "Column extra should have value";
-  EXPECT_EQ(valExt, *(result->Value("d", "ext"))) << "Column ext should have value";
-
-  // delete all cells from "extra" column
-  table->Delete(hbase::Delete{row}.AddColumns("d", "extra"));
-  result = table->Get(get);
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  ASSERT_FALSE(result->Value("d", "1")) << "Column 1 should be gone";
-  ASSERT_FALSE(result->Value("d", "extra")) << "Column extra should be gone";
-  EXPECT_EQ(valExt, *(result->Value("d", "ext"))) << "Column ext should have value";
-
-  // Delete the row and verify that subsequent Get returns nothing
-  table->Delete(hbase::Delete{row}.AddFamily("d"));
-  result = table->Get(get);
-  ASSERT_TRUE(result->IsEmpty()) << "Result should be empty.";
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, Increment) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("t1", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t1");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-  int64_t incr1 = 1235;
-  auto result = table->Increment(hbase::Increment{row}.AddColumn("d", "1", incr1));
-  EXPECT_EQ(row, result->Row());
-
-  long l = hbase::BytesUtil::ToInt64(*(result->Value("d", "1")));
-  EXPECT_EQ(incr1, l);
-
-  int64_t incr2 = -2;
-  result = table->Increment(hbase::Increment{row}.AddColumn("d", "1", incr2));
-
-  EXPECT_EQ(row, result->Row());
-  EXPECT_EQ(incr1 + incr2, hbase::BytesUtil::ToInt64(*(result->Value("d", "1"))));
-}
-
-TEST_F(ClientTest, CheckAndPut) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("check", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("check");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  // Perform Puts
-  table->Put(Put{row}.AddColumn("d", "1", "value1"));
-  auto result = table->CheckAndPut(row, "d", "1", "value1", Put{row}.AddColumn("d", "1", "value2"));
-  ASSERT_TRUE(result) << "CheckAndPut didn't replace value";
-
-  result = table->CheckAndPut(row, "d", "1", "value1", Put{row}.AddColumn("d", "1", "value3"));
-
-  // Perform the Get
-  hbase::Get get(row);
-  auto result1 = table->Get(get);
-  EXPECT_EQ("value2", *(result1->Value("d", "1")));
-  ASSERT_FALSE(result) << "CheckAndPut shouldn't replace value";
-}
-
-TEST_F(ClientTest, CheckAndDelete) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("checkDel", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("checkDel");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  auto val1 = "value1";
-
-  // Perform Puts
-  table->Put(Put{row}.AddColumn("d", "1", val1));
-  table->Put(Put{row}.AddColumn("d", "2", "value2"));
-  auto result = table->CheckAndDelete(row, "d", "1", val1, hbase::Delete{row}.AddColumn("d", "2"));
-  ASSERT_TRUE(result) << "CheckAndDelete didn't replace value";
-
-  // Perform the Get
-  hbase::Get get(row);
-  auto result1 = table->Get(get);
-  EXPECT_EQ(val1, *(result1->Value("d", "1")));
-  ASSERT_FALSE(result1->Value("d", "2")) << "Column 2 should be gone";
-}
-
-TEST_F(ClientTest, PutGet) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("t", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  // Perform Puts
-  table->Put(Put{"test1"}.AddColumn("d", "1", "value1"));
-  table->Put(Put{"test1"}.AddColumn("d", "extra", "value for extra"));
-
-  // Perform the Get
-  hbase::Get get(row);
-  auto result = table->Get(get);
-
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ("test1", result->Row());
-  EXPECT_EQ("value1", *(result->Value("d", "1")));
-  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, GetForNonExistentTable) {
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t_not_exists");
-  auto row = "test1";
-
-  // Get to be performed on above HBase Table
-  hbase::Get get(row);
-
-  ClientTest::test_util->conf()->SetInt("hbase.client.retries.number", 5);
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  // Perform the Get
-  try {
-    table->Get(get);
-    FAIL() << "Should have thrown RetriesExhaustedException";
-  } catch (const RetriesExhaustedException &ex) {
-    ASSERT_EQ(0, ex.num_retries());
-  } catch (...) {
-    FAIL() << "Should have thrown RetriesExhaustedException";
-  }
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, GetForNonExistentRow) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("t_exists", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t_exists");
-  auto row = "row_not_exists";
-
-  // Get to be performed on above HBase Table
-  hbase::Get get(row);
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  // Perform the Get
-  auto result = table->Get(get);
-  ASSERT_TRUE(result->IsEmpty()) << "Result should  be empty.";
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, PutsWithTimestamp) {
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable("t_puts_with_timestamp", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t_puts_with_timestamp");
-  auto row = "test1";
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  int64_t ts = 42;
-  // Perform Puts
-  table->Put(Put{"test1"}.AddColumn("d", "1", ts, "value1"));
-  auto cell =
-      std::make_unique<Cell>("test1", "d", "extra", ts, "value for extra", hbase::CellType::PUT);
-  table->Put(Put{"test1"}.Add(std::move(cell)));
-
-  // Perform the Get
-  hbase::Get get(row);
-  auto result = table->Get(get);
-
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ("test1", result->Row());
-  EXPECT_EQ("value1", *(result->Value("d", "1")));
-  EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
-  EXPECT_EQ(ts, result->ColumnLatestCell("d", "1")->Timestamp());
-  EXPECT_EQ(ts, result->ColumnLatestCell("d", "extra")->Timestamp());
-
-  table->Close();
-  client.Close();
-}
-
-void SetClientParams() {
-  ClientTest::test_util->conf()->SetInt("hbase.client.cpu.thread.pool.size", 6);
-  ClientTest::test_util->conf()->SetInt("hbase.client.operation.timeout", 600000);
-  ClientTest::test_util->conf()->SetInt("hbase.client.retries.number", 7);
-  ClientTest::test_util->conf()->SetInt("hbase.client.start.log.errors.counter", 1);
-}
-
-void PerformPuts(uint64_t num_rows, std::shared_ptr<hbase::Client> client,
-                 const std::string &table_name) {
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-  auto table = client->Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-  // Perform Puts
-  for (uint64_t i = 0; i < num_rows; i++) {
-    table->Put(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
-                                                         "value" + std::to_string(i)));
-  }
-}
-
-void MakeGets(uint64_t num_rows, const std::string &row_prefix, std::vector<hbase::Get> &gets) {
-  // Perform the Gets
-  for (uint64_t i = 0; i < num_rows; ++i) {
-    auto row = "test" + std::to_string(i);
-    hbase::Get get(row);
-    gets.push_back(get);
-  }
-  gets.push_back(hbase::Get("test2"));
-  gets.push_back(hbase::Get("testextra"));
-}
-
-void TestMultiResults(uint64_t num_rows, const std::vector<std::shared_ptr<hbase::Result>> &results,
-                      const std::vector<hbase::Get> &gets) {
-  // Test the values, should be same as in put executed on hbase shell
-  ASSERT_TRUE(!results.empty()) << "Result vector shouldn't be empty.";
-
-  uint32_t i = 0;
-  for (; i < num_rows; ++i) {
-    ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row()
-                                        << " must not be empty";
-    EXPECT_EQ("test" + std::to_string(i), results[i]->Row());
-    EXPECT_EQ("value" + std::to_string(i), results[i]->Value("d", std::to_string(i)).value());
-  }
-  // We are inserting test2 twice so the below test should pass
-  ASSERT_TRUE(!results[i]->IsEmpty()) << "Result for Get " << gets[i].row() << " must not be empty";
-
-  ++i;
-  ASSERT_TRUE(results[i]->IsEmpty()) << "Result for Get " << gets[i].row() << " must be empty";
-}
-
-TEST_F(ClientTest, MultiGets) {
-  std::string table_name = "t";
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable(table_name, "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  uint64_t num_rows = 50000;
-  PerformPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  std::vector<hbase::Get> gets;
-  MakeGets(num_rows, "test", gets);
-
-  auto results = table->Get(gets);
-
-  TestMultiResults(num_rows, results, gets);
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, MultiGetsWithRegionSplits) {
-  // Using TestUtil to populate test data
-  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
-                                "test500", "test600", "test700", "test800", "test900"};
-  std::string table_name = "t";
-  ClientTest::test_util->CreateTable(table_name, "d", keys);
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  uint64_t num_rows = 50000;
-  PerformPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table != nullptr) << "Unable to get connection to Table.";
-
-  std::vector<hbase::Get> gets;
-  MakeGets(num_rows, "test", gets);
-
-  auto results = table->Get(gets);
-
-  TestMultiResults(num_rows, results, gets);
-
-  table->Close();
-  client.Close();
-}
-
-void PerformMultiPuts(uint64_t num_rows, std::shared_ptr<hbase::Client> client,
-                      const std::string &table_name) {
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-  auto table = client->Table(tn);
-  ASSERT_TRUE(table) << "Unable to get connection to Table.";
-  std::vector<hbase::Put> puts;
-  // Perform Puts
-  for (uint64_t i = 0; i < num_rows; i++) {
-    puts.push_back(Put{"test" + std::to_string(i)}.AddColumn("d", std::to_string(i),
-                                                             "value" + std::to_string(i)));
-  }
-  table->Put(puts);
-}
-
-void PerformMultiPuts(std::vector<hbase::Put> &puts, std::shared_ptr<Table> table) {
-  table->Put(puts);
-}
-
-TEST_F(ClientTest, MultiGetsWithMultiPuts) {
-  std::string table_name = "t";
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable(table_name, "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  uint64_t num_rows = 50000;
-  PerformMultiPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table) << "Unable to get connection to Table.";
-
-  std::vector<hbase::Get> gets;
-  MakeGets(num_rows, "test", gets);
-
-  auto results = table->Get(gets);
-
-  TestMultiResults(num_rows, results, gets);
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, MultiGetsWithMultiPutsAndSplitRegions) {
-  // Using TestUtil to populate test data
-  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
-                                "test500", "test600", "test700", "test800", "test900"};
-  std::string table_name = "t";
-  ClientTest::test_util->CreateTable(table_name, "d", keys);
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-
-  uint64_t num_rows = 50000;
-  PerformMultiPuts(num_rows, std::make_shared<hbase::Client>(client), table_name);
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-  ASSERT_TRUE(table) << "Unable to get connection to Table.";
-
-  std::vector<hbase::Get> gets;
-  MakeGets(num_rows, "test", gets);
-
-  auto results = table->Get(gets);
-
-  TestMultiResults(num_rows, results, gets);
-
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, MultiPuts) {
-  std::string table_name = "t";
-  // Using TestUtil to populate test data
-  ClientTest::test_util->CreateTable(table_name, "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-  std::shared_ptr<Table> table = client.Table(tn);
-  ASSERT_TRUE(table) << "Unable to get connection to Table.";
-
-  uint64_t num_rows = 80000;
-  uint64_t batch_num_rows = 10000;
-  std::vector<hbase::Put> puts;
-  for (uint64_t i = 0; i < num_rows;) {
-    puts.clear();
-    // accumulate batch_num_rows at a time
-    for (uint64_t j = 0; j < batch_num_rows && i < num_rows; ++j) {
-      hbase::Put put("test" + std::to_string(i));
-      put.AddColumn("d", std::to_string(i), "value" + std::to_string(i));
-      puts.push_back(put);
-      i++;
-    }
-    PerformMultiPuts(puts, table);
-  }
-  table->Close();
-  client.Close();
-}
-
-TEST_F(ClientTest, MultiPutsWithRegionSplits) {
-  // Using TestUtil to populate test data
-  std::vector<std::string> keys{"test0",   "test100", "test200", "test300", "test400",
-                                "test500", "test600", "test700", "test800", "test900"};
-  std::string table_name = "t";
-  ClientTest::test_util->CreateTable(table_name, "d", keys);
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>(table_name);
-
-  SetClientParams();
-
-  // Create a client
-  hbase::Client client(*ClientTest::test_util->conf());
-  std::shared_ptr<Table> table = client.Table(tn);
-  ASSERT_TRUE(table) << "Unable to get connection to Table.";
-
-  uint64_t num_rows = 80000;
-  uint64_t batch_num_rows = 10000;
-  std::vector<hbase::Put> puts;
-  for (uint64_t i = 0; i < num_rows;) {
-    puts.clear();
-    // accumulate batch_num_rows at a time
-    for (uint64_t j = 0; j < batch_num_rows && i < num_rows; ++j) {
-      hbase::Put put("test" + std::to_string(i));
-      put.AddColumn("d", std::to_string(i), "value" + std::to_string(i));
-      puts.push_back(put);
-      i++;
-    }
-    PerformMultiPuts(puts, table);
-  }
-  table->Close();
-  client.Close();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/client.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/client.cc b/hbase-native-client/core/client.cc
deleted file mode 100644
index e23aeae..0000000
--- a/hbase-native-client/core/client.cc
+++ /dev/null
@@ -1,55 +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/client.h"
-
-#include <glog/logging.h>
-#include <chrono>
-#include <exception>
-#include <memory>
-#include <utility>
-
-using hbase::pb::TableName;
-
-namespace hbase {
-
-Client::Client() {
-  HBaseConfigurationLoader loader;
-  auto conf = loader.LoadDefaultResources();
-  if (!conf) {
-    LOG(ERROR) << "Unable to create default Configuration object. Either hbase-default.xml or "
-                  "hbase-site.xml is absent in the search path or problems in XML parsing";
-    throw std::runtime_error("Configuration object not present.");
-  }
-  Init(conf.value());
-}
-
-Client::Client(const Configuration &conf) { Init(conf); }
-
-void Client::Init(const Configuration &conf) {
-  auto conf_ = std::make_shared<Configuration>(conf);
-  async_connection_ = AsyncConnectionImpl::Create(conf_);
-}
-
-std::unique_ptr<Table> Client::Table(const TableName &table_name) {
-  return std::make_unique<hbase::Table>(table_name, async_connection_);
-}
-
-void Client::Close() { async_connection_->Close(); }
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/client.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/client.h b/hbase-native-client/core/client.h
deleted file mode 100644
index 5563a15..0000000
--- a/hbase-native-client/core/client.h
+++ /dev/null
@@ -1,77 +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.
- *
- */
-
-#pragma once
-
-#include <memory>
-#include <string>
-
-#include "connection/rpc-client.h"
-#include "core/async-connection.h"
-#include "core/configuration.h"
-
-#include "core/table.h"
-#include "serde/table-name.h"
-
-namespace hbase {
-
-class Table;
-/**
- * Client.
- *
- * This is the class that provides access to an HBase cluster.
- * It is thread safe and does connection pooling. Current recommendations are to
- * have only one Client per cluster around.
- */
-class Client {
- public:
-  /**
-   * @brief Create a new client.
-   * @param quorum_spec Where to connect to get Zookeeper bootstrap information.
-   */
-  Client();
-  explicit Client(const Configuration& conf);
-  ~Client() = default;
-
-  /**
-   * @brief Retrieve a Table implementation for accessing a table.
-   * @param - table_name
-   */
-  std::unique_ptr<::hbase::Table> Table(const pb::TableName& table_name);
-
-  /**
-   * @brief Close the Client connection.
-   */
-  void Close();
-
-  /**
-   * @brief Internal. DO NOT USE.
-   */
-  std::shared_ptr<AsyncConnectionImpl> async_connection() { return async_connection_; }
-
- private:
-  /** Data */
-  std::shared_ptr<AsyncConnectionImpl> async_connection_;
-
- private:
-  /** Methods */
-  void Init(const Configuration& conf);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/configuration-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/configuration-test.cc b/hbase-native-client/core/configuration-test.cc
deleted file mode 100644
index abdf0c7..0000000
--- a/hbase-native-client/core/configuration-test.cc
+++ /dev/null
@@ -1,119 +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/configuration.h"
-#include <gtest/gtest.h>
-
-using hbase::Configuration;
-
-TEST(Configuration, SetGetBool) {
-  Configuration conf;
-
-  /* test true/false */
-  conf.SetBool("bool_key1", true);
-  EXPECT_EQ(true, conf.GetBool("bool_key1", false));
-  conf.SetBool("bool_key2", false);
-  EXPECT_EQ(false, conf.GetBool("bool_key2", true));
-
-  /* test 1/0 */
-  conf.SetBool("bool_key3", 1);
-  EXPECT_EQ(true, conf.GetBool("bool_key3", false));
-  conf.SetBool("bool_key4", 0);
-  EXPECT_EQ(false, conf.GetBool("bool_key4", true));
-
-  /* test non zero integer */
-  conf.SetBool("bool_key5", 5);
-  EXPECT_EQ(true, conf.GetBool("bool_key5", false));
-  conf.SetBool("bool_key6", -1);
-  EXPECT_EQ(true, conf.GetBool("bool_key5", false));
-
-  /* test non zero float */
-  conf.SetBool("bool_key7", 5.1);
-  EXPECT_EQ(true, conf.GetBool("bool_key7", false));
-  conf.SetBool("bool_key8", -1.2);
-  EXPECT_EQ(true, conf.GetBool("bool_key8", false));
-}
-
-TEST(Configuration, SetGetForBool) {
-  Configuration conf;
-
-  /* test true/false */
-  conf.Set("bool_key1", "true");
-  EXPECT_EQ(true, conf.GetBool("bool_key1", false));
-  conf.Set("bool_key2", "false");
-  EXPECT_EQ(false, conf.GetBool("bool_key2", true));
-
-  /* test 1/0 */
-  conf.Set("bool_key3", "1");
-  EXPECT_EQ(true, conf.GetBool("bool_key3", false));
-  conf.Set("bool_key4", "0");
-  EXPECT_EQ(false, conf.GetBool("bool_key4", true));
-
-  /* test non zero integer */
-  conf.Set("bool_key5", "5");
-  EXPECT_THROW(conf.GetBool("bool_key5", false), std::runtime_error);
-  conf.Set("bool_key6", "-1");
-  EXPECT_THROW(conf.GetBool("bool_key6", false), std::runtime_error);
-
-  /* test non zero float */
-  conf.Set("bool_key7", "5.1");
-  EXPECT_THROW(conf.GetBool("bool_key7", false), std::runtime_error);
-  conf.Set("bool_key8", "-1.2");
-  EXPECT_THROW(conf.GetBool("bool_key8", false), std::runtime_error);
-}
-
-TEST(Configuration, SetGet) {
-  Configuration conf;
-
-  EXPECT_EQ(conf.Get("foo", "default"), "default");
-  conf.Set("foo", "bar");
-  EXPECT_EQ(conf.Get("foo", "default"), "bar");
-}
-
-TEST(Configuration, SetGetInt) {
-  Configuration conf;
-
-  EXPECT_EQ(conf.GetInt("foo", 0), 0);
-  conf.SetInt("foo", 42);
-  EXPECT_EQ(conf.GetInt("foo", 0), 42);
-}
-
-TEST(Configuration, SetGetLong) {
-  Configuration conf;
-
-  EXPECT_EQ(conf.GetLong("foo", 0), 0);
-  conf.SetLong("foo", 42);
-  EXPECT_EQ(conf.GetLong("foo", 0), 42);
-}
-
-TEST(Configuration, SetGetDouble) {
-  Configuration conf;
-
-  EXPECT_EQ(conf.GetDouble("foo", 0), 0);
-  conf.SetDouble("foo", 42.0);
-  EXPECT_EQ(conf.GetDouble("foo", 0), 42.0);
-}
-
-TEST(Configuration, SetGetBoolBasic) {
-  Configuration conf;
-
-  EXPECT_EQ(conf.GetBool("foo", false), false);
-  conf.SetInt("foo", true);
-  EXPECT_EQ(conf.GetInt("foo", false), true);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/configuration.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/configuration.cc b/hbase-native-client/core/configuration.cc
deleted file mode 100644
index 1fd2851..0000000
--- a/hbase-native-client/core/configuration.cc
+++ /dev/null
@@ -1,244 +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/configuration.h"
-
-#include <memory>
-#include <stdexcept>
-#include <utility>
-
-#include <glog/logging.h>
-#include <boost/format.hpp>
-#include <boost/lexical_cast.hpp>
-
-namespace hbase {
-
-Configuration::Configuration() : hb_property_() {}
-
-Configuration::Configuration(ConfigMap &config_map) : hb_property_(std::move(config_map)) {}
-
-Configuration::~Configuration() {}
-
-size_t Configuration::IsSubVariable(const std::string &expr, std::string &sub_variable) const {
-  size_t start_pos = expr.find("${");
-  if (std::string::npos != start_pos) {
-    size_t pos_next = expr.find("}", start_pos + 1);
-    if (std::string::npos != pos_next) {
-      sub_variable = expr.substr(start_pos + 2, pos_next - (start_pos + 2));
-    }
-  }
-  return start_pos;
-}
-
-std::string Configuration::SubstituteVars(const std::string &expr) const {
-  if (0 == expr.size()) return expr;
-
-  std::string eval(expr);
-  std::string value_to_be_replaced("");
-  std::string var("");
-  for (int i = 0; i < kMaxSubsts; i++) {
-    var = "";
-    size_t start_pos = IsSubVariable(eval, var);
-    if (start_pos != std::string::npos) {
-      // We are blindly checking for environment property at first.
-      // If we don't get any value from GetEnv, check in hbase-site.xml.
-      value_to_be_replaced = GetEnv(var).value_or(GetProperty(var).value_or(""));
-
-      // we haven't found any value yet so we are returning eval
-      if (0 == value_to_be_replaced.size()) {
-        return eval;
-      }
-
-      // return original expression if there is a loop
-      if (value_to_be_replaced == expr) {
-        return expr;
-      }
-
-      eval.replace(start_pos, var.size() + 3, value_to_be_replaced);
-
-    } else {
-      // No further expansion required.
-      return eval;
-    }
-  }
-  // We reached here if the loop is exhausted
-  // If MAX_SUBSTS is exhausted, check if more variable substitution is reqd.
-  // If any-more substitutions are reqd, throw an error.
-  var = "";
-  if (IsSubVariable(eval, var) != std::string::npos) {
-    throw std::runtime_error("Variable substitution depth too large: " +
-                             std::to_string(kMaxSubsts) + " " + expr);
-  } else {
-    return eval;
-  }
-}
-
-optional<std::string> Configuration::GetEnv(const std::string &key) const {
-  char buf[2048];
-
-  if ("user.name" == key) {
-#ifdef HAVE_GETLOGIN
-    return std::experimental::make_optional(getlogin());
-#else
-    DLOG(WARNING) << "Client user.name not implemented";
-    return optional<std::string>();
-#endif
-  }
-
-  if ("user.dir" == key) {
-#ifdef HAVE_GETCWD
-    if (getcwd(buf, sizeof(buf))) {
-      return std::experimental::make_optional(buf);
-    } else {
-      return optional<std::string>();
-    }
-#else
-    DLOG(WARNING) << "Client user.dir not implemented";
-    return optional<std::string>();
-#endif
-  }
-
-  if ("user.home" == key) {
-#if defined(HAVE_GETUID) && defined(HAVE_GETPWUID_R)
-    uid = getuid();
-    if (!getpwuid_r(uid, &pw, buf, sizeof(buf), &pwp)) {
-      return std::experimental::make_optional(buf);
-    } else {
-      return optional<std::string>();
-    }
-#else
-    DLOG(WARNING) << "Client user.home not implemented";
-    return optional<std::string>();
-#endif
-  }
-  return optional<std::string>();
-}
-
-optional<std::string> Configuration::GetProperty(const std::string &key) const {
-  auto found = hb_property_.find(key);
-  if (found != hb_property_.end()) {
-    return std::experimental::make_optional(found->second.value);
-  } else {
-    return optional<std::string>();
-  }
-}
-
-optional<std::string> Configuration::Get(const std::string &key) const {
-  optional<std::string> raw = GetProperty(key);
-  if (raw) {
-    return std::experimental::make_optional(SubstituteVars(*raw));
-  } else {
-    return optional<std::string>();
-  }
-}
-
-std::string Configuration::Get(const std::string &key, const std::string &default_value) const {
-  return Get(key).value_or(default_value);
-}
-
-optional<int32_t> Configuration::GetInt(const std::string &key) const {
-  optional<std::string> raw = Get(key);
-  if (raw) {
-    try {
-      return std::experimental::make_optional(boost::lexical_cast<int32_t>(*raw));
-    } catch (const boost::bad_lexical_cast &blex) {
-      throw std::runtime_error(blex.what());
-    }
-  }
-  return optional<int32_t>();
-}
-
-int32_t Configuration::GetInt(const std::string &key, int32_t default_value) const {
-  return GetInt(key).value_or(default_value);
-}
-
-optional<int64_t> Configuration::GetLong(const std::string &key) const {
-  optional<std::string> raw = Get(key);
-  if (raw) {
-    try {
-      return std::experimental::make_optional(boost::lexical_cast<int64_t>(*raw));
-    } catch (const boost::bad_lexical_cast &blex) {
-      throw std::runtime_error(blex.what());
-    }
-  }
-  return optional<int64_t>();
-}
-
-int64_t Configuration::GetLong(const std::string &key, int64_t default_value) const {
-  return GetLong(key).value_or(default_value);
-}
-
-optional<double> Configuration::GetDouble(const std::string &key) const {
-  optional<std::string> raw = Get(key);
-  if (raw) {
-    try {
-      return std::experimental::make_optional(boost::lexical_cast<double>(*raw));
-    } catch (const boost::bad_lexical_cast &blex) {
-      throw std::runtime_error(blex.what());
-    }
-  }
-  return optional<double>();
-}
-
-double Configuration::GetDouble(const std::string &key, double default_value) const {
-  return GetDouble(key).value_or(default_value);
-}
-
-optional<bool> Configuration::GetBool(const std::string &key) const {
-  optional<std::string> raw = Get(key);
-  if (raw) {
-    if (!strcasecmp((*raw).c_str(), "true") || !strcasecmp((*raw).c_str(), "1")) {
-      return std::experimental::make_optional(true);
-    } else if (!strcasecmp((*raw).c_str(), "false") || !strcasecmp((*raw).c_str(), "0")) {
-      return std::experimental::make_optional(false);
-    } else {
-      boost::format what("Unexpected value \"%s\" found being converted to bool for key \"%s\"");
-      what % (*raw);
-      what % key;
-      throw std::runtime_error(what.str());
-    }
-  }
-  return optional<bool>();
-}
-
-bool Configuration::GetBool(const std::string &key, bool default_value) const {
-  return GetBool(key).value_or(default_value);
-}
-
-void Configuration::Set(const std::string &key, const std::string &value) {
-  hb_property_[key] = value;
-}
-
-void Configuration::SetInt(const std::string &key, int32_t value) {
-  Set(key, boost::lexical_cast<std::string>(value));
-}
-
-void Configuration::SetLong(const std::string &key, int64_t value) {
-  Set(key, boost::lexical_cast<std::string>(value));
-}
-
-void Configuration::SetDouble(const std::string &key, double value) {
-  Set(key, boost::lexical_cast<std::string>(value));
-}
-
-void Configuration::SetBool(const std::string &key, bool value) {
-  Set(key, boost::lexical_cast<std::string>(value));
-}
-
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/configuration.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/configuration.h b/hbase-native-client/core/configuration.h
deleted file mode 100644
index d70941c..0000000
--- a/hbase-native-client/core/configuration.h
+++ /dev/null
@@ -1,232 +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.
- *
- */
-
-/* Some pieces of code have been added from HDFS-8707 */
-#pragma once
-
-#include <map>
-#include <string>
-#include <vector>
-
-#include <experimental/optional>
-
-namespace hbase {
-
-template <class T>
-using optional = std::experimental::optional<T>;
-
-class Configuration {
- public:
-  /**
-   * @brief Create Configuration object using an empty map. This
-   * object WILL NOT be initialized from reading XML configuration (hbase-site.xml, etc).
-   * Use HBaseConfigurationLoader to initialize the configuration from
-   * hbase-default.xml and hbase-site.xml files.
-   */
-  Configuration();
-
-  ~Configuration();
-
-  /**
-   * @brief Returns value identified by key in ConfigMap else default value if
-   * property is absent.
-   * @param key Property whose value is to be fetched. SubstituteVars will be
-   * called for any variable expansion.
-   */
-  std::string Get(const std::string &key, const std::string &default_value) const;
-
-  /**
-   * @brief Returns int32_t identified by key in ConfigMap else default value if
-   * property is absent.
-   * @param key Property whose value is to be fetched. Internally Get(key) will
-   * be called.
-   * @throws std::runtime_error if conversion to int32_t fails
-   */
-  int32_t GetInt(const std::string &key, int32_t default_value) const;
-
-  /**
-   * @brief Returns int64_t identified by key in ConfigMap else default value if
-   * property is absent.
-   * @param key Property whose value is to be fetched. Internally Get(key) will
-   * be called and
-   * @throws std::runtime_error if conversion to int64_t fails
-   */
-  int64_t GetLong(const std::string &key, int64_t default_value) const;
-
-  /**
-   * @brief Returns double identified by key in ConfigMap else default value if
-   * property is absent.
-   * @param key Property whose value is to be fetched. Internally Get(key) will
-   * be called.
-   * @throws std::runtime_error if conversion to double fails
-   */
-  double GetDouble(const std::string &key, double default_value) const;
-
-  /**
-   * @brief Returns bool identified by key in ConfigMap else default value if
-   * property is absent.
-   * @param key Property whose value is to be fetched. Internally Get(key) will
-   * be called.
-   * @throws std::runtime_error if conversion to bool fails
-   */
-  bool GetBool(const std::string &key, bool default_value) const;
-
-  /**
-   * @brief This method sets the given key to the value.
-   * @param key property name
-   * @param value property value
-   */
-  void Set(const std::string &key, const std::string &value);
-
-  /**
-   * @brief This method sets the given key to the value.
-   * @param key property name
-   * @param value property value
-   */
-  void SetInt(const std::string &key, int32_t value);
-
-  /**
-   * @brief This method sets the given key to the value.
-   * @param key property name
-   * @param value property value
-   */
-  void SetLong(const std::string &key, int64_t value);
-
-  /**
-   * @brief This method sets the given key to the value.
-   * @param key property name
-   * @param value property value
-   */
-  void SetDouble(const std::string &key, double value);
-
-  /**
-   * @brief This method sets the given key to the value.
-   * @param key property name
-   * @param value property value
-   */
-  void SetBool(const std::string &key, bool value);
-
- private:
-  friend class HBaseConfigurationLoader;
-
-  /* Transparent data holder for property values */
-  /* Same as Jira HDFS-8707 */
-  struct ConfigData {
-    std::string value;
-    bool final;
-    ConfigData() : final(false) {}
-    explicit ConfigData(const std::string &value) : value(value), final(false) {}
-    void operator=(const std::string &new_value) {
-      value = new_value;
-      final = false;
-    }
-  };
-
-  /**
-   * @brief Map which hold configuration properties.
-   */
-  using ConfigMap = std::map<std::string, ConfigData>;
-
-  /**
-   * @brief Create Configuration object using config_map;
-   * @param config_map - Map consisting of properties.
-   */
-  explicit Configuration(ConfigMap &config_map);
-
-  // Property map filled all the properties loaded by ConfigurationLoader
-  ConfigMap hb_property_;
-
-  // Variable expansion depth
-  const int kMaxSubsts = 20;
-
-  /**
-   * @brief returns value for a property identified by key in ConfigMap.
-   * SubstituteVars will be called for any variable expansion.
-   * @param key Key whose value is to be fetched.
-   */
-  optional<std::string> Get(const std::string &key) const;
-
-  /**
-   * @brief returns optional int32_t value identified by the key in ConfigMap.
-   * Get(key) is called to get the string value which is then converted to
-   * int32_t using boost::lexical_cast.
-   * @param key Key whose value is to be fetched.
-   * @throws std::runtime_error if conversion to int32_t fails
-   */
-  optional<int32_t> GetInt(const std::string &key) const;
-
-  /**
-   * @brief returns optional int64_t value identified by the key in ConfigMap.
-   * Get(key) is called internally to get the string value which is then
-   * converted to int64_t using boost::lexical_cast.
-   * @param key Key whose value is to be fetched.
-   * @throws std::runtime_error if conversion to int64_t fails
-   */
-  optional<int64_t> GetLong(const std::string &key) const;
-
-  /**
-   * @brief returns optional double value identified by the key in ConfigMap.
-   * Get(key) is called to get the string value which is then converted to
-   * double using boost::lexical_cast.
-   * @param key Key whose value is to be fetched.
-   * @throws std::runtime_error if conversion to double fails
-   */
-  optional<double> GetDouble(const std::string &key) const;
-
-  /**
-   * @brief returns optional bool for a property identified by key in ConfigMap.
-   * Get(key) is called to get the string value which is then converted to bool
-   * by checking the validity.
-   * @param key Key whose value is to be fetched. Get(key) is called to get the
-   * string value which is then converted to bool.
-   * @throws std::runtime_error if conversion to bool fails
-   */
-  optional<bool> GetBool(const std::string &key) const;
-
-  /**
-   * @brief This method will perform any variable expansion if present.
-   * @param expression expression string to perform substitutions on.
-   * @throws std::runtime_error if MAX_SUBSTS is exhausted and any more
-   * substitutions are reqd to be performed.
-   */
-  std::string SubstituteVars(const std::string &expr) const;
-
-  /**
-   * @brief This method will check if variable expansion has to be performed or
-   * not. Expression will be checked for presence of "${" and "}" to perform
-   * variable expansion.
-   * @param expr Expression on which will be checked for validity.
-   * @param sub_variable Extracted variable from expr which will be checked
-   * against environment value or ConfigMap values.
-   */
-  size_t IsSubVariable(const std::string &expr, std::string &sub_variable) const;
-
-  /**
-   * @brief This method will fetch value for key from environment if present.
-   * @param key key to be fetched from environment.
-   */
-  optional<std::string> GetEnv(const std::string &key) const;
-
-  /**
-   * @brief This method will fetch value for key from ConfigMap if present.
-   * @param key key to be fetched from environment.
-   */
-  optional<std::string> GetProperty(const std::string &key) const;
-};
-} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/connection-configuration.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/connection-configuration.h b/hbase-native-client/core/connection-configuration.h
deleted file mode 100644
index 995798e..0000000
--- a/hbase-native-client/core/connection-configuration.h
+++ /dev/null
@@ -1,207 +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.
- *
- */
-
-#pragma once
-
-#include <chrono>
-#include <climits>
-#include <string>
-
-#include "core/configuration.h"
-
-namespace hbase {
-
-/**
- * Timeout configs.
- */
-class ConnectionConfiguration {
- public:
-  explicit ConnectionConfiguration(const Configuration& conf) {
-    connect_timeout_ =
-        ToNanos(conf.GetInt(kClientSocketConnectTimeout, kDefaultClientSocketConnectTimeout));
-    meta_operation_timeout_ =
-        ToNanos(conf.GetLong(kClientMetaOperationTimeout, kDefaultClientOperationTimeout));
-    operation_timeout_ =
-        ToNanos(conf.GetLong(kClientOperationTimeout, kDefaultClientOperationTimeout));
-    rpc_timeout_ = ToNanos(conf.GetLong(kRpcTimeout, kDefaultRpcTimeout));
-    read_rpc_timeout_ = ToNanos(conf.GetLong(kRpcReadTimeout, ToMillis(rpc_timeout_)));
-    write_rpc_timeout_ = ToNanos(conf.GetLong(kRpcWriteTimeout, ToMillis(rpc_timeout_)));
-    pause_ = ToNanos(conf.GetLong(kClientPause, kDefaultClientPause));
-    max_retries_ = conf.GetInt(kClientRetriesNumber, kDefaultClientRetriesNumber);
-    start_log_errors_count_ =
-        conf.GetInt(kStartLogErrorsAfterCount, kDefaultStartLogErrorsAfterCount);
-    scan_timeout_ =
-        ToNanos(conf.GetLong(kClientScannerTimeoutPeriod, kDefaultClientScannerTimeoutPeriod));
-    scanner_caching_ = conf.GetInt(kClientScannerCaching, kDefaultClientScannerCaching);
-    scanner_max_result_size_ =
-        conf.GetLong(kClientScannerMaxResultsSize, kDefaultClientScannerMaxResultsSize);
-  }
-
-  // Used by tests
-  ConnectionConfiguration(std::chrono::nanoseconds connect_timeout,
-                          std::chrono::nanoseconds operation_timeout,
-                          std::chrono::nanoseconds rpc_timeout, std::chrono::nanoseconds pause,
-                          uint32_t max_retries, uint32_t start_log_errors_count)
-      : connect_timeout_(connect_timeout),
-        operation_timeout_(operation_timeout),
-        meta_operation_timeout_(operation_timeout),
-        rpc_timeout_(rpc_timeout),
-        read_rpc_timeout_(rpc_timeout),
-        write_rpc_timeout_(rpc_timeout),
-        pause_(pause),
-        max_retries_(max_retries),
-        start_log_errors_count_(start_log_errors_count) {}
-
-  std::chrono::nanoseconds connect_timeout() const { return connect_timeout_; }
-
-  std::chrono::nanoseconds meta_operation_timeout() const { return meta_operation_timeout_; }
-
-  // timeout for a whole operation such as get, put or delete. Notice that scan will not be effected
-  // by this value, see scanTimeoutNs.
-  std::chrono::nanoseconds operation_timeout() const { return operation_timeout_; }
-
-  // timeout for each rpc request. Can be overridden by a more specific config, such as
-  // readRpcTimeout or writeRpcTimeout.
-  std::chrono::nanoseconds rpc_timeout() const { return rpc_timeout_; }
-
-  // timeout for each read rpc request
-  std::chrono::nanoseconds read_rpc_timeout() const { return read_rpc_timeout_; }
-
-  // timeout for each write rpc request
-  std::chrono::nanoseconds write_rpc_timeout() const { return write_rpc_timeout_; }
-
-  std::chrono::nanoseconds pause() const { return pause_; }
-
-  uint32_t max_retries() const { return max_retries_; }
-
-  /** How many retries are allowed before we start to log */
-  uint32_t start_log_errors_count() const { return start_log_errors_count_; }
-
-  // The scan timeout is used as operation timeout for every
-  // operations in a scan, such as openScanner or next.
-  std::chrono::nanoseconds scan_timeout() const { return scan_timeout_; }
-
-  uint32_t scanner_caching() const { return scanner_caching_; }
-
-  uint64_t scanner_max_result_size() const { return scanner_max_result_size_; }
-
- private:
-  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
-  static constexpr const char* kClientSocketConnectTimeout =
-      "hbase.ipc.client.socket.timeout.connect";
-  /** Parameter name for HBase client CPU thread pool size. Defaults to (2 * num cpus) */
-  static constexpr const uint32_t kDefaultClientSocketConnectTimeout = 10000;  // 10 secs
-
-  /** Parameter name for HBase client operation timeout. */
-  static constexpr const char* kClientOperationTimeout = "hbase.client.operation.timeout";
-
-  /** Parameter name for HBase client meta operation timeout. */
-  static constexpr const char* kClientMetaOperationTimeout = "hbase.client.meta.operation.timeout";
-
-  /** Default HBase client operation timeout, which is tantamount to a blocking call */
-  static constexpr const uint32_t kDefaultClientOperationTimeout = 1200000;
-
-  /** timeout for each RPC */
-  static constexpr const char* kRpcTimeout = "hbase.rpc.timeout";
-
-  /** timeout for each read RPC */
-  static constexpr const char* kRpcReadTimeout = "hbase.rpc.read.timeout";
-
-  /** timeout for each write RPC */
-  static constexpr const char* kRpcWriteTimeout = "hbase.rpc.write.timeout";
-
-  static constexpr const uint32_t kDefaultRpcTimeout = 60000;
-
-  /**
-    * Parameter name for client pause value, used mostly as value to wait
-    * before running a retry of a failed get, region lookup, etc.
-    */
-  static constexpr const char* kClientPause = "hbase.client.pause";
-
-  static constexpr const uint64_t kDefaultClientPause = 100;
-
-  /**
-   * Parameter name for maximum retries, used as maximum for all retryable
-   * operations such as fetching of the root region from root region server,
-   * getting a cell's value, starting a row update, etc.
-   */
-  static constexpr const char* kClientRetriesNumber = "hbase.client.retries.number";
-
-  static constexpr const uint32_t kDefaultClientRetriesNumber = 35;
-
-  /**
-    * Configure the number of failures after which the client will start logging. A few failures
-    * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable
-    * heuristic for the number of errors we don't log. 9 was chosen because we wait for 1s at
-    * this stage.
-    */
-  static constexpr const char* kStartLogErrorsAfterCount = "hbase.client.start.log.errors.counter";
-  static constexpr const uint32_t kDefaultStartLogErrorsAfterCount = 9;
-
-  /** The client scanner timeout period in milliseconds. */
-  static constexpr const char* kClientScannerTimeoutPeriod = "hbase.client.scanner.timeout.period";
-
-  static constexpr const uint32_t kDefaultClientScannerTimeoutPeriod = 60000;
-
-  /**
-   * Parameter name to set the default scanner caching for all clients.
-   */
-  static constexpr const char* kClientScannerCaching = "hbase.client.scanner.caching";
-
-  static constexpr const uint32_t kDefaultClientScannerCaching = INT_MAX;
-
-  /**
-   * Parameter name for maximum number of bytes returned when calling a scanner's next method.
-   * Controlled by the client.
-   */
-  static constexpr const char* kClientScannerMaxResultsSize =
-      "hbase.client.scanner.max.result.size";
-
-  /**
-   * Maximum number of bytes returned when calling a scanner's next method.
-   * Note that when a single row is larger than this limit the row is still
-   * returned completely.
-   *
-   * The default value is 2MB.
-   */
-  static constexpr const uint64_t kDefaultClientScannerMaxResultsSize = 2 * 1024 * 1024;
-
-  std::chrono::nanoseconds connect_timeout_;
-  std::chrono::nanoseconds meta_operation_timeout_;
-  std::chrono::nanoseconds operation_timeout_;
-  std::chrono::nanoseconds rpc_timeout_;
-  std::chrono::nanoseconds read_rpc_timeout_;
-  std::chrono::nanoseconds write_rpc_timeout_;
-  std::chrono::nanoseconds pause_;
-  uint32_t max_retries_;
-  uint32_t start_log_errors_count_;
-  std::chrono::nanoseconds scan_timeout_;
-  uint32_t scanner_caching_;
-  uint64_t scanner_max_result_size_;
-
-  static std::chrono::nanoseconds ToNanos(const uint64_t& millis) {
-    return std::chrono::duration_cast<std::chrono::nanoseconds>(std::chrono::milliseconds(millis));
-  }
-
-  static uint64_t ToMillis(const std::chrono::nanoseconds& nanos) {
-    return std::chrono::duration_cast<std::chrono::milliseconds>(nanos).count();
-  }
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/delete-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/delete-test.cc b/hbase-native-client/core/delete-test.cc
deleted file mode 100644
index ec1e3a9..0000000
--- a/hbase-native-client/core/delete-test.cc
+++ /dev/null
@@ -1,124 +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 <glog/logging.h>
-#include <gtest/gtest.h>
-
-#include "core/delete.h"
-#include "core/mutation.h"
-#include "utils/time-util.h"
-
-using hbase::Delete;
-using hbase::Cell;
-using hbase::CellType;
-using hbase::Mutation;
-using hbase::TimeUtil;
-
-const constexpr int64_t Mutation::kLatestTimestamp;
-
-TEST(Delete, Row) {
-  Delete del{"foo"};
-  EXPECT_EQ("foo", del.row());
-}
-
-TEST(Delete, Timestamp) {
-  Delete del{"row"};
-
-  // test default timestamp
-  EXPECT_EQ(Mutation::kLatestTimestamp, del.TimeStamp());
-
-  // set custom timestamp
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  del.SetTimeStamp(ts);
-  EXPECT_EQ(ts, del.TimeStamp());
-
-  // Add a column with custom timestamp
-  del.AddColumn("f", "q");
-  auto &cell = del.FamilyMap().at("f")[0];
-  EXPECT_EQ(ts, cell->Timestamp());
-}
-
-TEST(Delete, HasFamilies) {
-  Delete del{"row"};
-
-  EXPECT_EQ(false, del.HasFamilies());
-
-  del.AddColumn("f", "q");
-  EXPECT_EQ(true, del.HasFamilies());
-}
-
-TEST(Delete, Add) {
-  CellType cell_type = CellType::DELETE;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  int64_t timestamp = std::numeric_limits<int64_t>::max();
-  auto cell = std::make_unique<Cell>(row, family, column, timestamp, "", cell_type);
-
-  // add first cell
-  Delete del{"row"};
-  del.Add(std::move(cell));
-  EXPECT_EQ(1, del.FamilyMap().size());
-  EXPECT_EQ(1, del.FamilyMap().at(family).size());
-
-  // add a non-matching row
-  auto cell2 = std::make_unique<Cell>(row, family, column, timestamp, "", cell_type);
-  Delete del2{"foo"};
-  ASSERT_THROW(del2.Add(std::move(cell2)), std::runtime_error);  // rows don't match
-
-  // add a second cell with same family
-  auto cell3 = std::make_unique<Cell>(row, family, "column-2", timestamp, "", cell_type);
-  del.Add(std::move(cell3));
-  EXPECT_EQ(1, del.FamilyMap().size());
-  EXPECT_EQ(2, del.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  auto cell4 = std::make_unique<Cell>(row, "family-2", "column-2", timestamp, "", cell_type);
-  del.Add(std::move(cell4));
-  EXPECT_EQ(2, del.FamilyMap().size());
-  EXPECT_EQ(1, del.FamilyMap().at("family-2").size());
-}
-
-TEST(Delete, AddColumn) {
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-
-  Delete del{"row"};
-  del.AddColumn(family, column);
-  EXPECT_EQ(1, del.FamilyMap().size());
-  EXPECT_EQ(1, del.FamilyMap().at(family).size());
-
-  // add a second cell with same family
-  del.AddColumn(family, "column-2");
-  EXPECT_EQ(1, del.FamilyMap().size());
-  EXPECT_EQ(2, del.FamilyMap().at(family).size());
-
-  // add a cell to a different family
-  del.AddColumn("family-2", column);
-  EXPECT_EQ(2, del.FamilyMap().size());
-  EXPECT_EQ(1, del.FamilyMap().at("family-2").size());
-
-  // use the AddColumn overload
-  auto ts = TimeUtil::ToMillis(TimeUtil::GetNowNanos());
-  del.AddColumn(family, column, ts);
-  EXPECT_EQ(2, del.FamilyMap().size());
-  EXPECT_EQ(3, del.FamilyMap().at(family).size());
-  auto &cell = del.FamilyMap().at(family)[2];
-  EXPECT_EQ(ts, cell->Timestamp());
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/delete.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/delete.cc b/hbase-native-client/core/delete.cc
deleted file mode 100644
index 5f48782..0000000
--- a/hbase-native-client/core/delete.cc
+++ /dev/null
@@ -1,131 +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/delete.h"
-#include <folly/Conv.h>
-#include <algorithm>
-#include <limits>
-#include <stdexcept>
-#include <utility>
-
-namespace hbase {
-
-/**
- * @brief Add the specified column to this Delete operation.
- * This is an expensive call in that on the server-side, it first does a
- * get to find the latest versions timestamp.  Then it adds a delete using
- * the fetched cells timestamp.
- *  @param family family name
- *  @param qualifier column qualifier
- */
-Delete& Delete::AddColumn(const std::string& family, const std::string& qualifier) {
-  return AddColumn(family, qualifier, timestamp_);
-}
-
-/**
- *  @brief Add the specified column to this Delete operation.
- *  @param family family name
- *  @param qualifier column qualifier
- *  @param timestamp version timestamp
- */
-Delete& Delete::AddColumn(const std::string& family, const std::string& qualifier,
-                          int64_t timestamp) {
-  if (timestamp < 0) {
-    throw std::runtime_error("Timestamp cannot be negative. ts=" +
-                             folly::to<std::string>(timestamp));
-  }
-
-  return Add(
-      std::make_unique<Cell>(row_, family, qualifier, timestamp, "", hbase::CellType::DELETE));
-}
-/**
- * Delete all versions of the specified column.
- * @param family family name
- * @param qualifier column qualifier
- */
-Delete& Delete::AddColumns(const std::string& family, const std::string& qualifier) {
-  return AddColumns(family, qualifier, timestamp_);
-}
-/**
- * Delete all versions of the specified column with a timestamp less than
- * or equal to the specified timestamp.
- * @param family family name
- * @param qualifier column qualifier
- * @param timestamp maximum version timestamp
- */
-Delete& Delete::AddColumns(const std::string& family, const std::string& qualifier,
-                           int64_t timestamp) {
-  if (timestamp < 0) {
-    throw std::runtime_error("Timestamp cannot be negative. ts=" +
-                             folly::to<std::string>(timestamp));
-  }
-
-  return Add(std::make_unique<Cell>(row_, family, qualifier, timestamp, "",
-                                    hbase::CellType::DELETE_COLUMN));
-}
-/**
- * Delete all versions of all columns of the specified family.
- * <p>
- * Overrides previous calls to deleteColumn and deleteColumns for the
- * specified family.
- * @param family family name
- */
-Delete& Delete::AddFamily(const std::string& family) { return AddFamily(family, timestamp_); }
-
-/**
- * Delete all columns of the specified family with a timestamp less than
- * or equal to the specified timestamp.
- * <p>
- * Overrides previous calls to deleteColumn and deleteColumns for the
- * specified family.
- * @param family family name
- * @param timestamp maximum version timestamp
- */
-Delete& Delete::AddFamily(const std::string& family, int64_t timestamp) {
-  const auto& it = family_map_.find(family);
-  if (family_map_.end() != it) {
-    it->second.clear();
-  } else {
-    family_map_[family];
-  }
-  return Add(
-      std::make_unique<Cell>(row_, family, "", timestamp, "", hbase::CellType::DELETE_FAMILY));
-}
-/**
- * Delete all columns of the specified family with a timestamp equal to
- * the specified timestamp.
- * @param family family name
- * @param timestamp version timestamp
- */
-Delete& Delete::AddFamilyVersion(const std::string& family, int64_t timestamp) {
-  return Add(std::make_unique<Cell>(row_, family, "", timestamp, "",
-                                    hbase::CellType::DELETE_FAMILY_VERSION));
-}
-Delete& Delete::Add(std::unique_ptr<Cell> cell) {
-  if (cell->Row() != row_) {
-    throw std::runtime_error("The row in " + cell->DebugString() +
-                             " doesn't match the original one " + row_);
-  }
-
-  family_map_[cell->Family()].push_back(std::move(cell));
-  return *this;
-}
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/delete.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/delete.h b/hbase-native-client/core/delete.h
deleted file mode 100644
index 9ebb5a6..0000000
--- a/hbase-native-client/core/delete.h
+++ /dev/null
@@ -1,111 +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.
- *
- */
-
-#pragma once
-
-#include <cstdint>
-#include <map>
-#include <memory>
-#include <string>
-#include <vector>
-#include "core/cell.h"
-#include "core/mutation.h"
-
-namespace hbase {
-
-class Delete : public Mutation {
- public:
-  /**
-   * Constructors
-   */
-  /*
-   * If no further operations are done, this will delete everything
-   * associated with the specified row (all versions of all columns in all
-   * families), with timestamp from current point in time to the past.
-   * Cells defining timestamp for a future point in time
-   * (timestamp > current time) will not be deleted.
-   */
-  explicit Delete(const std::string& row) : Mutation(row) {}
-
-  Delete(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
-  Delete(const Delete& cdelete) : Mutation(cdelete) {}
-  Delete& operator=(const Delete& cdelete) {
-    Mutation::operator=(cdelete);
-    return *this;
-  }
-
-  ~Delete() = default;
-
-  /**
-   *  @brief Add the specified column to this Delete operation.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   */
-  Delete& AddColumn(const std::string& family, const std::string& qualifier);
-
-  /**
-   *  @brief Add the specified column to this Delete operation.
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param timestamp version timestamp
-   */
-  Delete& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp);
-
-  /**
-   *  @brief Deletes all versions of the specified column
-   *  @param family family name
-   *  @param qualifier column qualifier
-   */
-  Delete& AddColumns(const std::string& family, const std::string& qualifier);
-  /**
-   *  @brief Deletes all versions of the specified column with a timestamp less than
-   * or equal to the specified timestamp
-   *  @param family family name
-   *  @param qualifier column qualifier
-   *  @param timestamp version timestamp
-   */
-  Delete& AddColumns(const std::string& family, const std::string& qualifier, int64_t timestamp);
-  /**
-   *  @brief Add the specified family to this Delete operation.
-   *  @param family family name
-   */
-  Delete& AddFamily(const std::string& family);
-
-  /**
-   *  @brief Deletes all columns of the specified family with a timestamp less than
-   * or equal to the specified timestamp
-   *  @param family family name
-   *  @param timestamp version timestamp
-   */
-  Delete& AddFamily(const std::string& family, int64_t timestamp);
-  /**
-   *  @brief Deletes all columns of the specified family with a timestamp
-   *   equal to the specified timestamp
-   *  @param family family name
-   *  @param timestamp version timestamp
-   */
-  Delete& AddFamilyVersion(const std::string& family, int64_t timestamp);
-  /**
-   * Advanced use only.
-   * Add an existing delete marker to this Delete object.
-   */
-  Delete& Add(std::unique_ptr<Cell> cell);
-};
-
-}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/core/filter-test.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/filter-test.cc b/hbase-native-client/core/filter-test.cc
deleted file mode 100644
index 7276dfb..0000000
--- a/hbase-native-client/core/filter-test.cc
+++ /dev/null
@@ -1,141 +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 <gtest/gtest.h>
-#include "core/client.h"
-#include "core/configuration.h"
-#include "core/get.h"
-#include "core/put.h"
-#include "core/result.h"
-#include "core/table.h"
-#include "if/Comparator.pb.h"
-#include "if/HBase.pb.h"
-#include "serde/table-name.h"
-#include "test-util/test-util.h"
-
-using hbase::Configuration;
-using hbase::Get;
-using hbase::Put;
-using hbase::FilterFactory;
-using hbase::Table;
-using hbase::TestUtil;
-using hbase::pb::CompareType;
-using hbase::ComparatorFactory;
-using hbase::Comparator;
-
-class FilterTest : public ::testing::Test {
- protected:
-  static void SetUpTestCase() {
-    test_util_ = std::make_unique<TestUtil>();
-    test_util_->StartMiniCluster(2);
-  }
-
-  static void TearDownTestCase() { test_util_.release(); }
-
-  virtual void SetUp() {}
-  virtual void TearDown() {}
-
-  static std::unique_ptr<TestUtil> test_util_;
-};
-
-std::unique_ptr<TestUtil> FilterTest::test_util_ = nullptr;
-
-TEST_F(FilterTest, GetWithColumnPrefixFilter) {
-  // write row1 with 3 columns (column_1, column_2, and foo_column)
-  FilterTest::test_util_->CreateTable("t", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t");
-  auto row = "row1";
-
-  // Gets to be performed on above HBase Table
-  Get get_all(row);  // expected to return all 3 columns
-  Get get_one(row);  // expected to return 1 column
-  Get get_two(row);  // expected to return 2 column
-
-  get_one.SetFilter(FilterFactory::ColumnPrefixFilter("foo_"));
-  get_two.SetFilter(FilterFactory::ColumnPrefixFilter("column_"));
-
-  // Create a client
-  hbase::Client client(*(FilterTest::test_util_->conf()));
-  auto table = client.Table(tn);
-
-  table->Put(Put{"row1"}.AddColumn("d", "column_1", "value1"));
-  table->Put(Put{"row1"}.AddColumn("d", "column_2", "value2"));
-  table->Put(Put{"row1"}.AddColumn("d", "foo_column", "value3"));
-
-  // Perform the Get
-  auto result_all = table->Get(get_all);
-  auto result_one = table->Get(get_one);
-  auto result_two = table->Get(get_two);
-
-  table->Close();
-  client.Close();
-
-  // Test the values
-  ASSERT_TRUE(!result_one->IsEmpty()) << "Result shouldn't be empty.";
-  ASSERT_TRUE(!result_two->IsEmpty()) << "Result shouldn't be empty.";
-  ASSERT_TRUE(!result_all->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ(row, result_one->Row());
-  EXPECT_EQ(row, result_two->Row());
-  EXPECT_EQ(row, result_all->Row());
-  EXPECT_EQ(1, result_one->Size());
-  EXPECT_EQ(2, result_two->Size());
-  EXPECT_EQ(3, result_all->Size());
-  EXPECT_EQ("value3", *(result_one->Value("d", "foo_column")));
-  EXPECT_EQ("value1", *(result_two->Value("d", "column_1")));
-  EXPECT_EQ("value2", *(result_two->Value("d", "column_2")));
-}
-
-TEST_F(FilterTest, GetWithQualifierFilter) {
-  // write row1 with 3 columns (a,b,c)
-  FilterTest::test_util_->CreateTable("t1", "d");
-
-  // Create TableName and Row to be fetched from HBase
-  auto tn = folly::to<hbase::pb::TableName>("t1");
-  auto row = "row1";
-
-  // Gets to be performed on above HBase Table
-  Get get(row);
-  get.SetFilter(FilterFactory::QualifierFilter(CompareType::GREATER_OR_EQUAL,
-                                               *ComparatorFactory::BinaryComparator("b")));
-
-  // Create a client
-  hbase::Client client(*(FilterTest::test_util_->conf()));
-
-  // Get connection to HBase Table
-  auto table = client.Table(tn);
-
-  table->Put(Put{"row1"}.AddColumn("d", "a", "value1"));
-  table->Put(Put{"row1"}.AddColumn("d", "b", "value2"));
-  table->Put(Put{"row1"}.AddColumn("d", "c", "value3"));
-
-  // Perform the Get
-  auto result = table->Get(get);
-
-  table->Close();
-  client.Close();
-
-  // Test the values
-  ASSERT_TRUE(!result->IsEmpty()) << "Result shouldn't be empty.";
-  EXPECT_EQ(row, result->Row());
-  EXPECT_EQ(2, result->Size());
-  EXPECT_EQ("value2", *(result->Value("d", "b")));
-  EXPECT_EQ("value3", *(result->Value("d", "c")));
-}