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

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

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 */