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/03/29 23:53:55 UTC

[1/2] hbase git commit: HBASE-15894 [C++] Put and Mutation objects

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14850 8c7a8b9da -> 66f8f36ec


HBASE-15894 [C++] Put and Mutation objects


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

Branch: refs/heads/HBASE-14850
Commit: 924a2731adc4ba168dbd9af955c57c8eb0211896
Parents: 8c7a8b9
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Mar 29 16:34:20 2017 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Mar 29 16:34:20 2017 -0700

----------------------------------------------------------------------
 hbase-native-client/core/BUCK                 |  8 +-
 hbase-native-client/core/cell-test.cc         | 18 +++-
 hbase-native-client/core/cell.cc              | 39 ++++++++-
 hbase-native-client/core/cell.h               |  6 +-
 hbase-native-client/core/get-test.cc          | 36 ++++----
 hbase-native-client/core/get.cc               |  4 +-
 hbase-native-client/core/get.h                | 14 +--
 hbase-native-client/core/mutation.cc          | 69 +++++++++++++++
 hbase-native-client/core/mutation.h           | 99 ++++++++++++++++++++++
 hbase-native-client/core/put.cc               | 67 +++++++++++++++
 hbase-native-client/core/put.h                | 68 +++++++++++++++
 hbase-native-client/core/request-converter.cc |  5 +-
 hbase-native-client/core/result.cc            |  4 +-
 hbase-native-client/core/result.h             |  7 +-
 hbase-native-client/core/scan-test.cc         | 28 +++---
 hbase-native-client/core/scan.cc              |  6 +-
 hbase-native-client/core/scan.h               | 14 +--
 hbase-native-client/core/table.h              |  1 +
 hbase-native-client/utils/BUCK                | 14 ++-
 hbase-native-client/utils/bytes-util-test.cc  | 49 +++++++++++
 hbase-native-client/utils/bytes-util.cc       | 53 ++++++++++++
 hbase-native-client/utils/bytes-util.h        | 45 ++++++++++
 22 files changed, 580 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 7483980..03fc0fc 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -34,7 +34,8 @@ cxx_library(
         # Once meta lookup works
         "meta-utils.h",
         "get.h",
-        "time-range.h",
+        "mutation.h",
+        "put.h",
         "scan.h",
         "result.h",
         "request-converter.h",
@@ -44,6 +45,7 @@ cxx_library(
         "async-rpc-retrying-caller-factory.h",
         "async-rpc-retrying-caller.h",
         "hbase-rpc-controller.h",
+        "time-range.h",
         "zk-util.h",
         "action.h",
         "multi-response.h",
@@ -60,13 +62,15 @@ cxx_library(
         "location-cache.cc",
         "meta-utils.cc",
         "get.cc",
-        "time-range.cc",
+        "mutation.cc",
+        "put.cc",
         "scan.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",

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/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
index fb4d033..2ec983b 100644
--- a/hbase-native-client/core/cell-test.cc
+++ b/hbase-native-client/core/cell-test.cc
@@ -31,7 +31,6 @@ TEST(CellTest, CellFailureTest) {
   std::string column = "column";
   std::string value = "value";
   int64_t timestamp = std::numeric_limits<int64_t>::max();
-  std::string tags = "";
   std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
   if (cell.get()) {
     EXPECT_NE("row-value", cell.get()->Row());
@@ -166,3 +165,20 @@ TEST(CellTest, CellRowFamilyColumnValueTest) {
     EXPECT_EQ(cell_type, cell.get()->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());
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/cell.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.cc b/hbase-native-client/core/cell.cc
index 4982406..14a8014 100644
--- a/hbase-native-client/core/cell.cc
+++ b/hbase-native-client/core/cell.cc
@@ -18,8 +18,12 @@
  */
 
 #include "core/cell.h"
+#include <climits>
 #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,
@@ -33,8 +37,6 @@ Cell::Cell(const std::string &row, const std::string &family, const std::string
       sequence_id_(0) {
   if (0 == row.size()) throw std::runtime_error("Row size should be greater than 0");
 
-  if (0 == family.size()) throw std::runtime_error("Column family size should be greater than 0");
-
   if (0 >= timestamp) throw std::runtime_error("Timestamp should be greater than 0");
 }
 
@@ -54,4 +56,37 @@ 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 MINIMUM:
+      return "MINIMUM";
+    case PUT:
+      return "PUT";
+    case DELETE:
+      return "DELETE";
+    case DELETE_COLUMN:
+      return "DELETE_COLUMN";
+    case DELETE_FAMILY:
+      return "DELETE_FAMILY";
+    case MAXIMUM:
+      return "MAXIMUM";
+    default:
+      return "UNKNOWN";
+  }
+}
+
 } /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/cell.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.h b/hbase-native-client/core/cell.h
index 5a39dc3..c062e45 100644
--- a/hbase-native-client/core/cell.h
+++ b/hbase-native-client/core/cell.h
@@ -28,7 +28,7 @@ enum CellType {
   MINIMUM = 0,
   PUT = 4,
   DELETE = 8,
-  DELETEFAMILYVERSION = 10,
+  DELETE_FAMILY_VERSION = 10,
   DELETE_COLUMN = 12,
   DELETE_FAMILY = 14,
   MAXIMUM = 255
@@ -46,6 +46,7 @@ class Cell {
   const std::string &Value() const;
   CellType Type() const;
   int64_t SequenceId() const;
+  std::string DebugString() const;
 
  private:
   std::string row_;
@@ -58,6 +59,9 @@ class Cell {
   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/924a2731/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
index 6ee2715..fafff39 100644
--- a/hbase-native-client/core/get-test.cc
+++ b/hbase-native-client/core/get-test.cc
@@ -29,20 +29,20 @@ void CheckFamilies(Get &get) {
   EXPECT_EQ(false, get.HasFamilies());
   get.AddFamily("family-1");
   EXPECT_EQ(true, get.HasFamilies());
-  EXPECT_EQ(1, get.Family().size());
-  for (const auto &family : get.Family()) {
+  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.Family().size());
+  EXPECT_EQ(1, get.FamilyMap().size());
   get.AddFamily("family-2");
-  EXPECT_EQ(2, get.Family().size());
+  EXPECT_EQ(2, get.FamilyMap().size());
   get.AddFamily("family-3");
-  EXPECT_EQ(3, get.Family().size());
+  EXPECT_EQ(3, get.FamilyMap().size());
   int i = 1;
-  for (const auto &family : get.Family()) {
+  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());
@@ -55,8 +55,8 @@ void CheckFamilies(Get &get) {
   get.AddColumn("family-1", "column-3");
   get.AddColumn("family-2", "column-X");
 
-  EXPECT_EQ(3, get.Family().size());
-  auto it = get.Family().begin();
+  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());
@@ -71,20 +71,20 @@ void CheckFamilies(Get &get) {
   EXPECT_STREQ("family-3", it->first.c_str());
   EXPECT_EQ(0, it->second.size());
   ++it;
-  EXPECT_EQ(it, get.Family().end());
+  EXPECT_EQ(it, get.FamilyMap().end());
 }
 
 void CheckFamiliesAfterCopy(Get &get) {
   EXPECT_EQ(true, get.HasFamilies());
-  EXPECT_EQ(3, get.Family().size());
+  EXPECT_EQ(3, get.FamilyMap().size());
   int i = 1;
-  for (const auto &family : get.Family()) {
+  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.Family().begin();
+  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());
@@ -99,7 +99,7 @@ void CheckFamiliesAfterCopy(Get &get) {
   EXPECT_STREQ("family-3", it->first.c_str());
   EXPECT_EQ(0, it->second.size());
   ++it;
-  EXPECT_EQ(it, get.Family().end());
+  EXPECT_EQ(it, get.FamilyMap().end());
 }
 
 void GetMethods(Get &get, const std::string &row) {
@@ -159,19 +159,19 @@ TEST(Get, SingleGet) {
   // 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.Family().size());
+  EXPECT_EQ(3, get.FamilyMap().size());
 
   Get getcp_fam(get);
   // address of family maps should be different.
-  EXPECT_NE(&(get.Family()), &(getcp_fam.Family()));
+  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.Family().size());
+  EXPECT_EQ(4, get.FamilyMap().size());
   // Verify the source object's family elements
-  auto it = get.Family().begin();
+  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());
@@ -191,7 +191,7 @@ TEST(Get, SingleGet) {
   EXPECT_STREQ("column-A", it->second[0].c_str());
   EXPECT_STREQ("column-B", it->second[1].c_str());
   ++it;
-  EXPECT_EQ(it, get.Family().end());
+  EXPECT_EQ(it, get.FamilyMap().end());
 
   // Verifying the copied object's families. It will remain unchanged and below
   // tests should pass

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/get.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.cc b/hbase-native-client/core/get.cc
index afeb429..52b2ec5 100644
--- a/hbase-native-client/core/get.cc
+++ b/hbase-native-client/core/get.cc
@@ -87,7 +87,9 @@ Get &Get::SetConsistency(hbase::pb::Consistency consistency) {
 
 bool Get::HasFamilies() const { return !family_map_.empty(); }
 
-const FamilyMap &Get::Family() const { return family_map_; }
+const std::map<std::string, std::vector<std::string>> &Get::FamilyMap() const {
+  return family_map_;
+}
 
 int Get::MaxVersions() const { return max_versions_; }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/get.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.h b/hbase-native-client/core/get.h
index e0be4e7..098635d 100644
--- a/hbase-native-client/core/get.h
+++ b/hbase-native-client/core/get.h
@@ -32,12 +32,6 @@
 using hbase::Row;
 namespace hbase {
 
-/**
- * @brief Map consisting of column families and qualifiers to be used for Get
- * operation
- */
-using FamilyMap = std::map<std::string, std::vector<std::string>>;
-
 class Get : public Row, public Query {
  public:
   /**
@@ -73,10 +67,10 @@ class Get : public Row, public Query {
   Get& SetCacheBlocks(bool cache_blocks);
 
   /**
-   * @brief Returns the Get family map (FamilyMap) for this Get operation. Used
+   * @brief Returns the Get family map for this Get operation. Used
    * for constructing Scan object with an already constructed Get
    */
-  const FamilyMap& Family() const;
+  const std::map<std::string, std::vector<std::string>>& FamilyMap() const;
 
   /**
    * @brief Returns the timerange for this Get
@@ -112,7 +106,7 @@ class Get : public Row, public Query {
   Get& AddColumn(const std::string& family, const std::string& qualifier);
 
   /**
-   * @brief Returns true if family map (FamilyMap) is non empty false otherwise
+   * @brief Returns true if family map is non empty false otherwise
    */
   bool HasFamilies() const;
 
@@ -131,7 +125,7 @@ class Get : public Row, public Query {
   int32_t max_versions_ = 1;
   bool cache_blocks_ = true;
   bool check_existence_only_ = false;
-  FamilyMap family_map_;
+  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>();
 };

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/mutation.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/mutation.cc b/hbase-native-client/core/mutation.cc
new file mode 100644
index 0000000..ab33105
--- /dev/null
+++ b/hbase-native-client/core/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 "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/924a2731/hbase-native-client/core/mutation.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/mutation.h b/hbase-native-client/core/mutation.h
new file mode 100644
index 0000000..83392e1
--- /dev/null
+++ b/hbase-native-client/core/mutation.h
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 the row for this operation
+   */
+  const std::string& row() const;
+
+  /**
+   * @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);
+
+ protected:
+  static const constexpr int64_t kLatestTimestamp = std::numeric_limits<int64_t>::max();
+  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/924a2731/hbase-native-client/core/put.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put.cc b/hbase-native-client/core/put.cc
new file mode 100644
index 0000000..b81051e
--- /dev/null
+++ b/hbase-native-client/core/put.cc
@@ -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.
+ *
+ */
+
+#include "core/put.h"
+#include <folly/Conv.h>
+#include <algorithm>
+#include <limits>
+#include <stdexcept>
+
+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, kLatestTimestamp, 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/924a2731/hbase-native-client/core/put.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put.h b/hbase-native-client/core/put.h
new file mode 100644
index 0000000..8a43dec
--- /dev/null
+++ b/hbase-native-client/core/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 "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/924a2731/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
index ff92b5c..c40a56e 100644
--- a/hbase-native-client/core/request-converter.cc
+++ b/hbase-native-client/core/request-converter.cc
@@ -47,6 +47,7 @@ std::unique_ptr<Request> RequestConverter::ToGetRequest(const Get &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;
 }
 
@@ -78,7 +79,7 @@ std::unique_ptr<Request> RequestConverter::ToScanRequest(const Scan &scan,
   }
 
   if (scan.HasFamilies()) {
-    for (const auto &family : scan.Family()) {
+    for (const auto &family : scan.FamilyMap()) {
       auto column = pb_scan->add_column();
       column->set_family(family.first);
       for (const auto &qualifier : family.second) {
@@ -137,7 +138,7 @@ std::unique_ptr<hbase::pb::Get> RequestConverter::ToGet(const Get &get) {
   }
   pb_get->set_row(get.row());
   if (get.HasFamilies()) {
-    for (const auto &family : get.Family()) {
+    for (const auto &family : get.FamilyMap()) {
       auto column = pb_get->add_column();
       column->set_family(family.first);
       for (const auto &qualifier : family.second) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.cc b/hbase-native-client/core/result.cc
index 4db3fca..eb28b32 100644
--- a/hbase-native-client/core/result.cc
+++ b/hbase-native-client/core/result.cc
@@ -91,8 +91,8 @@ int Result::Size() const { return cells_.size(); }
 
 const ResultMap &Result::Map() const { return result_map_; }
 
-const ResultFamilyMap Result::FamilyMap(const std::string &family) const {
-  ResultFamilyMap family_map;
+const std::map<std::string, std::string> Result::FamilyMap(const std::string &family) const {
+  std::map<std::string, std::string> family_map;
   if (!IsEmpty()) {
     for (auto itr = result_map_.begin(); itr != result_map_.end(); ++itr) {
       if (family == itr->first) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.h b/hbase-native-client/core/result.h
index 8ff4311..8064197 100644
--- a/hbase-native-client/core/result.h
+++ b/hbase-native-client/core/result.h
@@ -38,11 +38,6 @@ using ResultMap =
     std::map<std::string,
              std::map<std::string, std::map<int64_t, std::string, std::greater<int64_t> > > >;
 
-/**
- * @brief Map of qualifiers to values.
- */
-using ResultFamilyMap = std::map<std::string, std::string>;
-
 class Result {
  public:
   /**
@@ -116,7 +111,7 @@ class Result {
    * Returns a Map of the form: Map<qualifier,value>
    * @param family - column family to get
    */
-  const ResultFamilyMap FamilyMap(const std::string &family) const;
+  const std::map<std::string, std::string> FamilyMap(const std::string &family) const;
 
  private:
   bool exists_ = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/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
index 9a26407..73fb6df 100644
--- a/hbase-native-client/core/scan-test.cc
+++ b/hbase-native-client/core/scan-test.cc
@@ -29,20 +29,20 @@ void CheckFamilies(Scan &scan) {
   EXPECT_EQ(false, scan.HasFamilies());
   scan.AddFamily("family-1");
   EXPECT_EQ(true, scan.HasFamilies());
-  EXPECT_EQ(1, scan.Family().size());
-  for (const auto &family : scan.Family()) {
+  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.Family().size());
+  EXPECT_EQ(1, scan.FamilyMap().size());
   scan.AddFamily("family-2");
-  EXPECT_EQ(2, scan.Family().size());
+  EXPECT_EQ(2, scan.FamilyMap().size());
   scan.AddFamily("family-3");
-  EXPECT_EQ(3, scan.Family().size());
+  EXPECT_EQ(3, scan.FamilyMap().size());
   int i = 1;
-  for (const auto &family : scan.Family()) {
+  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());
@@ -55,8 +55,8 @@ void CheckFamilies(Scan &scan) {
   scan.AddColumn("family-1", "column-3");
   scan.AddColumn("family-2", "column-X");
 
-  EXPECT_EQ(3, scan.Family().size());
-  auto it = scan.Family().begin();
+  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());
@@ -71,20 +71,20 @@ void CheckFamilies(Scan &scan) {
   EXPECT_STREQ("family-3", it->first.c_str());
   EXPECT_EQ(0, it->second.size());
   ++it;
-  EXPECT_EQ(it, scan.Family().end());
+  EXPECT_EQ(it, scan.FamilyMap().end());
 }
 
 void CheckFamiliesAfterCopy(Scan &scan) {
   EXPECT_EQ(true, scan.HasFamilies());
-  EXPECT_EQ(3, scan.Family().size());
+  EXPECT_EQ(3, scan.FamilyMap().size());
   int i = 1;
-  for (const auto &family : scan.Family()) {
+  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.Family().begin();
+  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());
@@ -99,7 +99,7 @@ void CheckFamiliesAfterCopy(Scan &scan) {
   EXPECT_STREQ("family-3", it->first.c_str());
   EXPECT_EQ(0, it->second.size());
   ++it;
-  EXPECT_EQ(it, scan.Family().end());
+  EXPECT_EQ(it, scan.FamilyMap().end());
 }
 
 void ScanMethods(Scan &scan) {
@@ -219,7 +219,7 @@ TEST(Scan, FromGet) {
   get.AddColumn("family-1", "column-3");
   get.AddColumn("family-2", "column-X");
 
-  EXPECT_EQ(3, get.Family().size());
+  EXPECT_EQ(3, get.FamilyMap().size());
 
   Scan scan(get);
   ScanMethods(scan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/scan.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.cc b/hbase-native-client/core/scan.cc
index 5335abd..5f315ec 100644
--- a/hbase-native-client/core/scan.cc
+++ b/hbase-native-client/core/scan.cc
@@ -75,7 +75,7 @@ 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.Family().begin(), get.Family().end());
+  family_map_.insert(get.FamilyMap().begin(), get.FamilyMap().end());
 }
 
 Scan &Scan::AddFamily(const std::string &family) {
@@ -191,5 +191,7 @@ void Scan::CheckRow(const std::string &row) {
 
 bool Scan::HasFamilies() const { return !family_map_.empty(); }
 
-const FamilyMap &Scan::Family() const { return family_map_; }
+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/924a2731/hbase-native-client/core/scan.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.h b/hbase-native-client/core/scan.h
index cedbe0a..fb302b7 100644
--- a/hbase-native-client/core/scan.h
+++ b/hbase-native-client/core/scan.h
@@ -30,12 +30,6 @@
 
 namespace hbase {
 
-/**
- * @brief Map consisting of column families and qualifiers to be used for Get
- * operation
- */
-using FamilyMap = std::map<std::string, std::vector<std::string>>;
-
 class Scan : public Query {
  public:
   /**
@@ -251,14 +245,14 @@ class Scan : public Query {
   const TimeRange &Timerange() const;
 
   /**
-   * @brief Returns true if family map (FamilyMap) is non empty false otherwise
+   * @brief Returns true if family map is non empty false otherwise
    */
   bool HasFamilies() const;
 
   /**
-   * @brief Returns the Scan family map (FamilyMap) for this Scan operation.
+   * @brief Returns the Scan family map for this Scan operation.
    */
-  const FamilyMap &Family() const;
+  const std::map<std::string, std::vector<std::string>> &FamilyMap() const;
 
  private:
   std::string start_row_ = "";
@@ -273,7 +267,7 @@ class Scan : public Query {
   bool allow_partial_results_ = false;
   hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
   std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
-  FamilyMap family_map_;
+  std::map<std::string, std::vector<std::string>> family_map_;
 
   /**
    * @brief Checks for row length validity, throws if length check fails,

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/core/table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/table.h b/hbase-native-client/core/table.h
index 93ab91b..803befe 100644
--- a/hbase-native-client/core/table.h
+++ b/hbase-native-client/core/table.h
@@ -30,6 +30,7 @@
 #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.h"
 #include "serde/table-name.h"

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/hbase-native-client/utils/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/utils/BUCK b/hbase-native-client/utils/BUCK
index eae929e..04e2b67 100644
--- a/hbase-native-client/utils/BUCK
+++ b/hbase-native-client/utils/BUCK
@@ -18,10 +18,14 @@
 cxx_library(
     name="utils",
     exported_headers=[
-        "user-util.h", "version.h", "connection-util.h", "sys-util.h",
-        "time-util.h"
+        "bytes-util.h",
+        "connection-util.h",
+        "sys-util.h",
+        "time-util.h",
+        "user-util.h",
+        "version.h",
     ],
-    srcs=["user-util.cc", "connection-util.cc"],
+    srcs=["bytes-util.cc", "connection-util.cc", "user-util.cc"],
     deps=['//third-party:folly',],
     tests=[":user-util-test"],
     visibility=['PUBLIC',],
@@ -30,3 +34,7 @@ cxx_test(
     name="user-util-test",
     srcs=["user-util-test.cc",],
     deps=[":utils",],)
+cxx_test(
+    name="bytes-util-test",
+    srcs=["bytes-util-test.cc",],
+    deps=[":utils",],)

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/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
new file mode 100644
index 0000000..d997ee2
--- /dev/null
+++ b/hbase-native-client/utils/bytes-util-test.cc
@@ -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.
+ *
+ */
+
+#include <folly/Logging.h>
+#include <gtest/gtest.h>
+#include <string>
+
+#include "utils/bytes-util.h"
+
+using namespace std;
+using namespace hbase;
+
+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"));
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/924a2731/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
new file mode 100644
index 0000000..5e4d728
--- /dev/null
+++ b/hbase-native-client/utils/bytes-util.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 "utils/bytes-util.h"
+
+#include <memory>
+#include <string>
+
+#include <glog/logging.h>
+
+namespace hbase {
+
+constexpr char BytesUtil::kHexChars[];
+
+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/924a2731/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
new file mode 100644
index 0000000..541b2d7
--- /dev/null
+++ b/hbase-native-client/utils/bytes-util.h
@@ -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.
+ *
+ */
+
+#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);
+};
+} /* namespace hbase */


[2/2] hbase git commit: HBASE-16365 [C++] End to end Table::Put()

Posted by en...@apache.org.
HBASE-16365 [C++] End to end Table::Put()


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

Branch: refs/heads/HBASE-14850
Commit: 66f8f36ec6247c8e6e2be301d25ed5138b2f57b3
Parents: 924a273
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Mar 29 16:53:45 2017 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Mar 29 16:53:45 2017 -0700

----------------------------------------------------------------------
 hbase-native-client/core/BUCK                   |   5 +
 .../core/async-rpc-retrying-test.cc             |   5 +-
 hbase-native-client/core/cell-test.cc           | 175 +++++++++----------
 hbase-native-client/core/cell.cc                |  22 +++
 hbase-native-client/core/cell.h                 |   2 +
 hbase-native-client/core/client-test.cc         | 119 ++++++++-----
 hbase-native-client/core/filter-test.cc         |  18 +-
 hbase-native-client/core/get-test.cc            |  14 +-
 hbase-native-client/core/mutation.h             |   9 +-
 hbase-native-client/core/put-test.cc            | 135 ++++++++++++++
 hbase-native-client/core/put.cc                 |   5 +-
 hbase-native-client/core/raw-async-table.cc     |  15 ++
 hbase-native-client/core/raw-async-table.h      |   5 +
 hbase-native-client/core/request-converter.cc   |  68 +++++++
 hbase-native-client/core/request-converter.h    |  16 +-
 hbase-native-client/core/result-test.cc         |   8 +-
 hbase-native-client/core/result.cc              |  22 +++
 hbase-native-client/core/result.h               |   2 +
 hbase-native-client/core/simple-client.cc       | 124 ++++++-------
 hbase-native-client/core/table.cc               |   5 +
 hbase-native-client/core/table.h                |   8 +
 hbase-native-client/test-util/mini-cluster.cc   |  19 --
 hbase-native-client/test-util/mini-cluster.h    |   3 -
 hbase-native-client/test-util/test-util.cc      |   6 +-
 hbase-native-client/test-util/test-util.h       |   2 -
 25 files changed, 548 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 03fc0fc..30c3390 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -130,6 +130,11 @@ cxx_test(
     deps=[":core",],
     run_test_separately=True,)
 cxx_test(
+    name="put-test",
+    srcs=["put-test.cc",],
+    deps=[":core",],
+    run_test_separately=True,)
+cxx_test(
     name="retry-test",
     srcs=["async-rpc-retrying-test.cc",],
     deps=[

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 5086286..3ed6866 100644
--- a/hbase-native-client/core/async-rpc-retrying-test.cc
+++ b/hbase-native-client/core/async-rpc-retrying-test.cc
@@ -163,8 +163,6 @@ TEST(AsyncRpcRetryTest, TestGetBasic) {
   test_util->StartMiniCluster(2);
 
   test_util->CreateTable("t", "d");
-  test_util->TablePut("t", "test2", "d", "2", "value2");
-  test_util->TablePut("t", "test2", "d", "extra", "value for extra");
 
   // Create TableName and Row to be fetched from HBase
   auto tn = folly::to<hbase::pb::TableName>("t");
@@ -180,6 +178,9 @@ TEST(AsyncRpcRetryTest, TestGetBasic) {
   auto table = client.Table(tn);
   ASSERT_TRUE(table) << "Unable to get connection to Table.";
 
+  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);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 2ec983b..efb835d 100644
--- a/hbase-native-client/core/cell-test.cc
+++ b/hbase-native-client/core/cell-test.cc
@@ -23,75 +23,65 @@
 #include <gtest/gtest.h>
 #include <memory>
 
-using namespace hbase;
-TEST(CellTest, CellFailureTest) {
-  CellType cell_type = CellType::PUT;
-  std::string row = "row";
-  std::string family = "family";
-  std::string column = "column";
-  std::string value = "value";
+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();
-  std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_NE("row-value", cell.get()->Row());
-    EXPECT_NE("family-value", cell.get()->Family());
-    EXPECT_NE("column-value", cell.get()->Qualifier());
-    EXPECT_NE("value-value", cell.get()->Value());
-    EXPECT_NE(8975431260, cell.get()->Timestamp());
-    EXPECT_NE(CellType::MAXIMUM, cell.get()->Type());
-  }
+  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, CellSuceessTest) {
+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;
-  const std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_EQ(row, cell.get()->Row());
-    EXPECT_EQ(family, cell.get()->Family());
-    EXPECT_EQ(column, cell.get()->Qualifier());
-    EXPECT_EQ(value, cell.get()->Value());
-    EXPECT_EQ(timestamp, cell.get()->Timestamp());
-    EXPECT_EQ(cell_type, cell.get()->Type());
-  }
+
+  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, MultipleCellsTest) {
-  std::vector<const Cell *> cells;
-  for (int i = 0; i < 5; i++) {
-    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();
-    row += std::to_string(i);
-    value += std::to_string(i);
-    CellType cell_type = CellType::PUT;
-    const Cell *cell = new Cell(row, family, column, timestamp, value, cell_type);
-    cells.push_back(cell);
-  }
-  int i = 0;
-  for (const auto cell : cells) {
-    std::string row = "row-value";
-    std::string value = "value-value";
-    row += std::to_string(i);
-    value += std::to_string(i);
-    EXPECT_EQ(row, cell->Row());
-    EXPECT_EQ("family-value", cell->Family());
-    EXPECT_EQ("column-value", cell->Qualifier());
-    EXPECT_EQ(value, cell->Value());
-    EXPECT_EQ(std::numeric_limits<int64_t>::max(), cell->Timestamp());
-    EXPECT_EQ(CellType::PUT, cell->Type());
-    i += 1;
-  }
-  for (const auto cell : cells) {
-    delete cell;
-  }
-  cells.clear();
+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) {
@@ -101,15 +91,14 @@ TEST(CellTest, CellRowTest) {
   std::string value = "";
   int64_t timestamp = std::numeric_limits<int64_t>::max();
   CellType cell_type = CellType::PUT;
-  std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_EQ(row, cell.get()->Row());
-    EXPECT_EQ(family, cell.get()->Family());
-    EXPECT_EQ(column, cell.get()->Qualifier());
-    EXPECT_EQ(value, cell.get()->Value());
-    EXPECT_EQ(timestamp, cell.get()->Timestamp());
-    EXPECT_EQ(cell_type, cell.get()->Type());
-  }
+  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) {
@@ -119,15 +108,14 @@ TEST(CellTest, CellRowFamilyTest) {
   std::string value = "";
   int64_t timestamp = std::numeric_limits<int64_t>::max();
   CellType cell_type = CellType::PUT;
-  const std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_EQ(row, cell.get()->Row());
-    EXPECT_EQ(family, cell.get()->Family());
-    EXPECT_EQ(column, cell.get()->Qualifier());
-    EXPECT_EQ(value, cell.get()->Value());
-    EXPECT_EQ(timestamp, cell.get()->Timestamp());
-    EXPECT_EQ(cell_type, cell.get()->Type());
-  }
+  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) {
@@ -137,15 +125,15 @@ TEST(CellTest, CellRowFamilyValueTest) {
   std::string value = "only-value";
   int64_t timestamp = std::numeric_limits<int64_t>::max();
   CellType cell_type = CellType::PUT;
-  const std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_EQ(row, cell.get()->Row());
-    EXPECT_EQ(family, cell.get()->Family());
-    EXPECT_EQ(column, cell.get()->Qualifier());
-    EXPECT_EQ(value, cell.get()->Value());
-    EXPECT_EQ(timestamp, cell.get()->Timestamp());
-    EXPECT_EQ(cell_type, cell.get()->Type());
-  }
+
+  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) {
@@ -155,15 +143,14 @@ TEST(CellTest, CellRowFamilyColumnValueTest) {
   std::string value = "only-value";
   int64_t timestamp = std::numeric_limits<int64_t>::max();
   CellType cell_type = CellType::PUT;
-  std::unique_ptr<Cell> cell(new Cell(row, family, column, timestamp, value, cell_type));
-  if (cell.get()) {
-    EXPECT_EQ(row, cell.get()->Row());
-    EXPECT_EQ(family, cell.get()->Family());
-    EXPECT_EQ(column, cell.get()->Qualifier());
-    EXPECT_EQ(value, cell.get()->Value());
-    EXPECT_EQ(timestamp, cell.get()->Timestamp());
-    EXPECT_EQ(cell_type, cell.get()->Type());
-  }
+  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) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/cell.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.cc b/hbase-native-client/core/cell.cc
index 14a8014..24788ab 100644
--- a/hbase-native-client/core/cell.cc
+++ b/hbase-native-client/core/cell.cc
@@ -19,6 +19,7 @@
 
 #include "core/cell.h"
 #include <climits>
+#include <limits>
 #include <stdexcept>
 
 #include "folly/Conv.h"
@@ -40,6 +41,27 @@ Cell::Cell(const std::string &row, const std::string &family, const std::string
   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_; }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/cell.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/cell.h b/hbase-native-client/core/cell.h
index c062e45..acedd96 100644
--- a/hbase-native-client/core/cell.h
+++ b/hbase-native-client/core/cell.h
@@ -38,6 +38,8 @@ 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 184a6a7..ff4879a 100644
--- a/hbase-native-client/core/client-test.cc
+++ b/hbase-native-client/core/client-test.cc
@@ -17,24 +17,47 @@
  *
  */
 
-#include "core/client.h"
 #include <gtest/gtest.h>
+
+#include "core/cell.h"
+#include "core/client.h"
 #include "core/configuration.h"
 #include "core/get.h"
 #include "core/hbase-configuration-loader.h"
+#include "core/put.h"
 #include "core/result.h"
 #include "core/table.h"
 #include "serde/table-name.h"
 #include "test-util/test-util.h"
 
+using hbase::Cell;
+using hbase::Configuration;
+using hbase::Get;
+using hbase::Put;
+using hbase::Table;
+using hbase::TestUtil;
+
 class ClientTest : public ::testing::Test {
  public:
-  const static std::string kDefHBaseConfPath;
-
-  const static std::string kHBaseDefaultXml;
-  const static std::string kHBaseSiteXml;
-
-  const static std::string kHBaseXmlData;
+  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;
@@ -55,7 +78,7 @@ class ClientTest : public ::testing::Test {
     // 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.c_str(), 1);
+    setenv("HBASE_CONF", kDefHBaseConfPath, 1);
   }
   static std::unique_ptr<hbase::TestUtil> test_util;
 
@@ -67,28 +90,6 @@ class ClientTest : public ::testing::Test {
 };
 std::unique_ptr<hbase::TestUtil> ClientTest::test_util = nullptr;
 
-const std::string ClientTest::kDefHBaseConfPath("./build/test-data/client-test/conf/");
-
-const std::string ClientTest::kHBaseDefaultXml("hbase-default.xml");
-const std::string ClientTest::kHBaseSiteXml("hbase-site.xml");
-
-const std::string ClientTest::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>");
-
 TEST_F(ClientTest, EmptyConfigurationPassedToClient) { ASSERT_ANY_THROW(hbase::Client client); }
 
 TEST_F(ClientTest, ConfigurationPassedToClient) {
@@ -114,18 +115,13 @@ TEST_F(ClientTest, DefaultConfiguration) {
   client.Close();
 }
 
-TEST_F(ClientTest, Get) {
+TEST_F(ClientTest, PutGet) {
   // Using TestUtil to populate test data
   ClientTest::test_util->CreateTable("t", "d");
-  ClientTest::test_util->TablePut("t", "test2", "d", "2", "value2");
-  ClientTest::test_util->TablePut("t", "test2", "d", "extra", "value for extra");
 
   // Create TableName and Row to be fetched from HBase
   auto tn = folly::to<hbase::pb::TableName>("t");
-  auto row = "test2";
-
-  // Get to be performed on above HBase Table
-  hbase::Get get(row);
+  auto row = "test1";
 
   // Create a client
   hbase::Client client(*ClientTest::test_util->conf());
@@ -134,13 +130,18 @@ TEST_F(ClientTest, Get) {
   auto table = client.Table(tn);
   ASSERT_TRUE(table) << "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("test2", result->Row());
-  EXPECT_EQ("value2", *(result->Value("d", "2")));
+  EXPECT_EQ("test1", result->Row());
+  EXPECT_EQ("value1", *(result->Value("d", "1")));
   EXPECT_EQ("value for extra", *(result->Value("d", "extra")));
 
   table->Close();
@@ -150,7 +151,7 @@ TEST_F(ClientTest, Get) {
 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 = "test2";
+  auto row = "test1";
 
   // Get to be performed on above HBase Table
   hbase::Get get(row);
@@ -194,3 +195,41 @@ TEST_F(ClientTest, GetForNonExistentRow) {
   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) << "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();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 40081cc..f401698 100644
--- a/hbase-native-client/core/filter-test.cc
+++ b/hbase-native-client/core/filter-test.cc
@@ -21,6 +21,7 @@
 #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"
@@ -30,6 +31,7 @@
 
 using hbase::Configuration;
 using hbase::Get;
+using hbase::Put;
 using hbase::FilterFactory;
 using hbase::Table;
 using hbase::TestUtil;
@@ -57,9 +59,6 @@ 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");
-  FilterTest::test_util_->TablePut("t", "row1", "d", "column_1", "value1");
-  FilterTest::test_util_->TablePut("t", "row1", "d", "column_2", "value2");
-  FilterTest::test_util_->TablePut("t", "row1", "d", "foo_column", "value3");
 
   // Create TableName and Row to be fetched from HBase
   auto tn = folly::to<hbase::pb::TableName>("t");
@@ -75,11 +74,13 @@ TEST_F(FilterTest, GetWithColumnPrefixFilter) {
 
   // Create a client
   hbase::Client client(*(FilterTest::test_util_->conf()));
-
-  // Get connection to HBase Table
   auto table = client.Table(tn);
   ASSERT_TRUE(table) << "Unable to get connection to Table.";
 
+  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);
@@ -106,9 +107,6 @@ TEST_F(FilterTest, GetWithColumnPrefixFilter) {
 TEST_F(FilterTest, GetWithQualifierFilter) {
   // write row1 with 3 columns (a,b,c)
   FilterTest::test_util_->CreateTable("t1", "d");
-  FilterTest::test_util_->TablePut("t1", "row1", "d", "a", "value1");
-  FilterTest::test_util_->TablePut("t1", "row1", "d", "b", "value2");
-  FilterTest::test_util_->TablePut("t1", "row1", "d", "c", "value3");
 
   // Create TableName and Row to be fetched from HBase
   auto tn = folly::to<hbase::pb::TableName>("t1");
@@ -126,6 +124,10 @@ TEST_F(FilterTest, GetWithQualifierFilter) {
   auto table = client.Table(tn);
   ASSERT_TRUE(table) << "Unable to get connection to Table.";
 
+  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);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index fafff39..6127e23 100644
--- a/hbase-native-client/core/get-test.cc
+++ b/hbase-native-client/core/get-test.cc
@@ -17,12 +17,15 @@
  *
  */
 
+#include "core/cell.h"
 #include "core/get.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) {
@@ -199,21 +202,16 @@ TEST(Get, SingleGet) {
 }
 
 TEST(Get, MultiGet) {
-  std::vector<Get *> gets;
+  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);
-    Get *get = new Get(row_str);
+    auto get = std::make_unique<Get>(row_str);
 
     GetMethods(*get, row_str);
-    gets.push_back(get);
+    gets.push_back(std::move(get));
   }
   EXPECT_EQ(NUMBER_OF_GETS, gets.size());
-
-  for (const auto &get : gets) {
-    delete get;
-  }
-  gets.clear();
 }
 
 TEST(Get, Exception) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/mutation.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/mutation.h b/hbase-native-client/core/mutation.h
index 83392e1..5e0381b 100644
--- a/hbase-native-client/core/mutation.h
+++ b/hbase-native-client/core/mutation.h
@@ -65,11 +65,6 @@ class Mutation: public Row {
   }
 
   /**
-   * @brief Returns the row for this operation
-   */
-  const std::string& row() const;
-
-  /**
    * @brief Returns true if family map is non empty false otherwise
    */
   bool HasFamilies() const;
@@ -85,8 +80,10 @@ class Mutation: public Row {
    */
   Mutation& SetDurability(pb::MutationProto_Durability durability);
 
- protected:
+ 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
new file mode 100644
index 0000000..d4ed00a
--- /dev/null
+++ b/hbase-native-client/core/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 "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/66f8f36e/hbase-native-client/core/put.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/put.cc b/hbase-native-client/core/put.cc
index b81051e..bb20c5c 100644
--- a/hbase-native-client/core/put.cc
+++ b/hbase-native-client/core/put.cc
@@ -24,6 +24,7 @@
 #include <algorithm>
 #include <limits>
 #include <stdexcept>
+#include <utility>
 
 namespace hbase {
 
@@ -35,7 +36,7 @@ namespace hbase {
  */
 Put& Put::AddColumn(const std::string& family, const std::string& qualifier,
                     const std::string& value) {
-  return AddColumn(family, qualifier, kLatestTimestamp, value);
+  return AddColumn(family, qualifier, timestamp_, value);
 }
 
 /**
@@ -57,7 +58,7 @@ Put& Put::AddColumn(const std::string& family, const std::string& qualifier, int
 
 Put& Put::Add(std::unique_ptr<Cell> cell) {
   if (cell->Row() != row_) {
-    throw std::runtime_error("The row in" + cell->DebugString() +
+    throw std::runtime_error("The row in " + cell->DebugString() +
                              " doesn't match the original one " + row_);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 9a680ed..2bc9f36 100644
--- a/hbase-native-client/core/raw-async-table.cc
+++ b/hbase-native-client/core/raw-async-table.cc
@@ -76,4 +76,19 @@ Future<std::shared_ptr<Result>> RawAsyncTable::Get(const hbase::Get& get) {
   return caller->Call().then([caller](const auto r) { return r; });
 }
 
+Future<Unit> RawAsyncTable::Put(const hbase::Put& put) {
+  auto caller =
+      CreateCallerBuilder<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<Unit> {
+            return Call<hbase::Put, hbase::Request, hbase::Response, Unit>(
+                rpc_client, controller, loc, put, &hbase::RequestConverter::ToMutateRequest,
+                [](const Response& r) -> Unit { return folly::unit; });
+          })
+          ->Build();
+
+  return caller->Call().then([caller](const auto r) { return r; });
+}
+
 } /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index bbdc6bd..978a2b8 100644
--- a/hbase-native-client/core/raw-async-table.h
+++ b/hbase-native-client/core/raw-async-table.h
@@ -19,6 +19,7 @@
 #pragma once
 
 #include <folly/futures/Future.h>
+#include <folly/futures/Unit.h>
 
 #include <chrono>
 #include <memory>
@@ -29,9 +30,11 @@
 #include "core/async-rpc-retrying-caller.h"
 #include "core/connection-configuration.h"
 #include "core/get.h"
+#include "core/put.h"
 #include "core/result.h"
 
 using folly::Future;
+using folly::Unit;
 using hbase::pb::TableName;
 using std::chrono::nanoseconds;
 using std::chrono::milliseconds;
@@ -52,6 +55,8 @@ class RawAsyncTable {
   virtual ~RawAsyncTable() = default;
 
   Future<std::shared_ptr<Result>> Get(const hbase::Get& get);
+
+  Future<Unit> Put(const hbase::Put& put);
   void Close() {}
 
  private:

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index c40a56e..4c12ee7 100644
--- a/hbase-native-client/core/request-converter.cc
+++ b/hbase-native-client/core/request-converter.cc
@@ -18,6 +18,9 @@
  */
 
 #include "core/request-converter.h"
+
+#include <folly/Conv.h>
+
 #include <utility>
 #include "if/Client.pb.h"
 
@@ -152,4 +155,69 @@ std::unique_ptr<hbase::pb::Get> RequestConverter::ToGet(const Get &get) {
   }
   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 DELETE:
+      return pb::MutationProto_DeleteType_DELETE_ONE_VERSION;
+    case DELETE_COLUMN:
+      return pb::MutationProto_DeleteType_DELETE_MULTIPLE_VERSIONS;
+    case DELETE_FAMILY:
+      return pb::MutationProto_DeleteType_DELETE_FAMILY;
+    case 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());
+
+  VLOG(3) << "Req is " << pb_req->req_msg()->ShortDebugString();
+  return pb_req;
+}
 } /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 003afaa..ff6b290 100644
--- a/hbase-native-client/core/request-converter.h
+++ b/hbase-native-client/core/request-converter.h
@@ -24,16 +24,23 @@
 #include <vector>
 #include "connection/request.h"
 #include "core/action.h"
+#include "core/cell.h"
 #include "core/get.h"
 #include "core/region-request.h"
 #include "core/scan.h"
 #include "core/server-request.h"
+#include "core/mutation.h"
+#include "core/put.h"
+#include "if/Client.pb.h"
 #include "if/HBase.pb.h"
 
-using hbase::pb::RegionSpecifier;
+using hbase::pb::MutationProto;
 using hbase::pb::RegionAction;
+using hbase::pb::RegionSpecifier;
 using hbase::pb::ServerName;
 using hbase::ServerRequest;
+using MutationType = hbase::pb::MutationProto_MutationType;
+using DeleteType = hbase::pb::MutationProto_DeleteType;
 
 namespace hbase {
 
@@ -64,6 +71,11 @@ class RequestConverter {
 
   static std::unique_ptr<Request> ToMultiRequest(const ActionsByRegion &region_requests);
 
+  static std::unique_ptr<Request> ToMutateRequest(const Put &put, const std::string &region_name);
+
+  static std::unique_ptr<MutationProto> ToMutation(const MutationType type,
+                                                   const Mutation &mutation, const int64_t nonce);
+
  private:
   // Constructor not required. We have all static methods to create PB requests.
   RequestConverter();
@@ -76,6 +88,8 @@ class RequestConverter {
    */
   static void SetRegion(const std::string &region_name, RegionSpecifier *region_specifier);
   static std::unique_ptr<hbase::pb::Get> ToGet(const Get &get);
+  static DeleteType ToDeleteType(const CellType type);
+  static bool IsDelete(const CellType type);
 };
 
 } /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 09d4e15..520f4b9 100644
--- a/hbase-native-client/core/result-test.cc
+++ b/hbase-native-client/core/result-test.cc
@@ -17,8 +17,6 @@
  *
  */
 
-#include "core/result.h"
-
 #include <gtest/gtest.h>
 #include <limits>
 #include <memory>
@@ -26,7 +24,11 @@
 #include <vector>
 
 #include "core/cell.h"
-using namespace hbase;
+#include "core/result.h"
+
+using hbase::Cell;
+using hbase::CellType;
+using hbase::Result;
 
 void PopulateCells(std::vector<std::shared_ptr<Cell> > &cells) {
   // Populate some Results

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/result.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.cc b/hbase-native-client/core/result.cc
index eb28b32..9d9ddb3 100644
--- a/hbase-native-client/core/result.cc
+++ b/hbase-native-client/core/result.cc
@@ -109,4 +109,26 @@ const std::map<std::string, std::string> Result::FamilyMap(const std::string &fa
   }
   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;
+}
+
 } /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/result.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/result.h b/hbase-native-client/core/result.h
index 8064197..627d161 100644
--- a/hbase-native-client/core/result.h
+++ b/hbase-native-client/core/result.h
@@ -113,6 +113,8 @@ class Result {
    */
   const std::map<std::string, std::string> FamilyMap(const std::string &family) const;
 
+  std::string DebugString() const;
+
  private:
   bool exists_ = false;
   bool stale_ = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 4b1144c..b417353 100644
--- a/hbase-native-client/core/simple-client.cc
+++ b/hbase-native-client/core/simple-client.cc
@@ -19,111 +19,91 @@
 
 #include <folly/Logging.h>
 #include <folly/Random.h>
-#include <folly/futures/Future.h>
 #include <gflags/gflags.h>
-#include <wangle/concurrent/CPUThreadPoolExecutor.h>
-#include <wangle/concurrent/IOThreadPoolExecutor.h>
 
 #include <atomic>
 #include <chrono>
 #include <iostream>
 #include <thread>
 
-#include "connection/connection-pool.h"
 #include "core/client.h"
-#include "core/keyvalue-codec.h"
-#include "if/Client.pb.h"
-#include "if/ZooKeeper.pb.h"
+#include "core/get.h"
+#include "core/put.h"
+#include "core/table.h"
 #include "serde/server-name.h"
 #include "serde/table-name.h"
+#include "utils/time-util.h"
 
-using namespace folly;
-using namespace std;
-using namespace std::chrono;
+using hbase::Client;
 using hbase::Configuration;
-using hbase::Response;
-using hbase::Request;
-using hbase::HBaseService;
-using hbase::KeyValueCodec;
-using hbase::LocationCache;
-using hbase::ConnectionPool;
-using hbase::ConnectionFactory;
+using hbase::Get;
+using hbase::Put;
+using hbase::Table;
 using hbase::pb::TableName;
 using hbase::pb::ServerName;
-using hbase::pb::RegionSpecifier_RegionSpecifierType;
-using hbase::pb::MutateRequest;
-using hbase::pb::MutationProto_MutationType;
+using hbase::TimeUtil;
 
-// TODO(eclark): remove the need for this.
-DEFINE_string(table, "t", "What region to send a get");
-DEFINE_string(row, "test", "What row to get");
+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_uint64(columns, 10000, "How many columns to write");
+DEFINE_uint64(num_rows, 10000, "How many rows to write and read");
+DEFINE_bool(display_results, false, "Whether to display the Results from Gets");
 DEFINE_int32(threads, 6, "How many cpu threads");
 
-std::unique_ptr<Request> MakeRequest(uint64_t col, std::string region_name) {
-  auto req = Request::mutate();
-  auto msg = std::static_pointer_cast<MutateRequest>(req->req_msg());
-  auto region = msg->mutable_region();
-  auto suf = folly::to<std::string>(col);
-
-  region->set_value(region_name);
-  region->set_type(
-      RegionSpecifier_RegionSpecifierType::RegionSpecifier_RegionSpecifierType_REGION_NAME);
-  auto mutation = msg->mutable_mutation();
-  mutation->set_row(FLAGS_row + suf);
-  mutation->set_mutate_type(MutationProto_MutationType::MutationProto_MutationType_PUT);
-  auto column = mutation->add_column_value();
-  column->set_family("d");
-  auto qual = column->add_qualifier_value();
-  qual->set_qualifier(suf);
-  qual->set_value(".");
-
-  return std::move(req);
+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;
 }
 
 int main(int argc, char *argv[]) {
   google::SetUsageMessage("Simple client to get a single row from HBase on the comamnd line");
   google::ParseCommandLineFlags(&argc, &argv, true);
   google::InitGoogleLogging(argv[0]);
-
-  // Set up thread pools.
-  auto cpu_pool = std::make_shared<wangle::CPUThreadPoolExecutor>(FLAGS_threads);
-  auto io_pool = std::make_shared<wangle::IOThreadPoolExecutor>(5);
-  auto codec = std::make_shared<KeyValueCodec>();
-  auto cp = std::make_shared<ConnectionPool>(io_pool, codec);
+  google::InstallFailureSignalHandler();
+  FLAGS_logtostderr = 1;
+  FLAGS_stderrthreshold = 1;
 
   // Configuration
   auto conf = std::make_shared<Configuration>();
   conf->Set("hbase.zookeeper.quorum", FLAGS_zookeeper);
-
-  // Create the cache.
-  LocationCache cache{conf, cpu_pool, cp};
+  conf->SetInt("hbase.client.cpu.thread.pool.size", FLAGS_threads);
 
   auto row = FLAGS_row;
-  auto tn = folly::to<TableName>(FLAGS_table);
-
-  auto loc = cache.LocateRegion(tn, row).get(milliseconds(5000));
-  auto connection = loc->service();
-
-  auto num_puts = FLAGS_columns;
-
-  auto results = std::vector<Future<std::unique_ptr<Response>>>{};
-  auto col = uint64_t{0};
-  for (; col < num_puts; col++) {
-    results.push_back(
-        folly::makeFuture(col)
-            .via(cpu_pool.get())
-            .then([loc](uint64_t col) { return MakeRequest(col, loc->region_name()); })
-            .then([connection](std::unique_ptr<Request> req) {
-              return (*connection)(std::move(req));
-            }));
+  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);
+
+  // Do the Put requests
+  auto start_ns = TimeUtil::GetNowNanos();
+  for (uint64_t i = 0; i < num_puts; i++) {
+    table->Put(*MakePut(Row(FLAGS_row, i)));
+  }
+
+  LOG(INFO) << "Successfully sent  " << num_puts << " Put requests in "
+            << TimeUtil::ElapsedMillis(start_ns) << " ms.";
+
+  // Do the Get requests
+  start_ns = TimeUtil::GetNowNanos();
+  for (uint64_t i = 0; i < num_puts; i++) {
+    auto result = table->Get(Get{Row(FLAGS_row, i)});
+    if (FLAGS_display_results) {
+      LOG(INFO) << result->DebugString();
+    }
   }
-  auto allf = folly::collect(results).get();
 
-  LOG(ERROR) << "Successfully sent  " << allf.size() << " requests.";
+  LOG(INFO) << "Successfully sent  " << num_puts << " Get requests in "
+            << TimeUtil::ElapsedMillis(start_ns) << " ms.";
 
-  io_pool->stop();
+  table->Close();
+  client->Close();
 
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/table.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/table.cc b/hbase-native-client/core/table.cc
index 3c54d78..8ace4af 100644
--- a/hbase-native-client/core/table.cc
+++ b/hbase-native-client/core/table.cc
@@ -54,6 +54,11 @@ std::shared_ptr<hbase::Result> Table::Get(const hbase::Get &get) {
   return context.get(operation_timeout());
 }
 
+void Table::Put(const hbase::Put &put) {
+  auto future = async_table_->Put(put);
+  future.get(operation_timeout());
+}
+
 milliseconds Table::operation_timeout() const {
   return TimeUtil::ToMillis(async_connection_->connection_conf()->operation_timeout());
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/hbase-native-client/core/table.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/table.h b/hbase-native-client/core/table.h
index 803befe..cbb95b7 100644
--- a/hbase-native-client/core/table.h
+++ b/hbase-native-client/core/table.h
@@ -58,6 +58,14 @@ class Table {
   // std::vector<std::unique_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);
+
+  // TODO: Batch Puts
+
+  /**
    * @brief - Close the client connection.
    */
   void Close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index fb58fc1..34da54c 100644
--- a/hbase-native-client/test-util/mini-cluster.cc
+++ b/hbase-native-client/test-util/mini-cluster.cc
@@ -239,25 +239,6 @@ jobject MiniCluster::admin() {
   return admin;
 }
 
-jobject MiniCluster::TablePut(const std::string &table, const std::string &row,
-                              const std::string &family, const std::string &column,
-                              const std::string &value) {
-  env();
-  jobject conn = env_->CallObjectMethod(htu(), get_conn_mid_);
-  jobject put = env_->NewObject(put_class_, put_ctor_, StrToByteChar(row));
-  if (put == NULL) {
-    LOG(INFO) << "Couldn't create Put";
-    exit(-1);
-  }
-  env_->CallObjectMethod(put, add_col_mid_, StrToByteChar(family), StrToByteChar(column),
-                         StrToByteChar(value));
-  jobject table_name_obj = env_->CallStaticObjectMethod(table_name_class_, tbl_name_value_of_mid_,
-                                                        env_->NewStringUTF(table.c_str()));
-  jobject table_obj = env_->CallObjectMethod(conn, get_table_mid_, table_name_obj);
-  env_->CallObjectMethod(table_obj, put_mid_, put);
-  return table_obj;
-}
-
 // moves region to server
 void MiniCluster::MoveRegion(const std::string &region, const std::string &server) {
   jobject admin_ = admin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index a9502b5..4119cb5 100644
--- a/hbase-native-client/test-util/mini-cluster.h
+++ b/hbase-native-client/test-util/mini-cluster.h
@@ -39,9 +39,6 @@ class MiniCluster {
   jobject GetConf();
   // returns the value for config key retrieved from cluster
   const std::string GetConfValue(const std::string &key);
-  // Does Put into table for family fam, qualifier col with value
-  jobject TablePut(const std::string &table, const std::string &row, const std::string &family,
-                   const std::string &column, const std::string &value);
 
  private:
   JNIEnv *env_;

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index 54d2c83..c4e6ed2 100644
--- a/hbase-native-client/test-util/test-util.cc
+++ b/hbase-native-client/test-util/test-util.cc
@@ -58,19 +58,17 @@ void TestUtil::StartMiniCluster(int32_t num_region_servers) {
   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::string &family,
                            const std::vector<std::string> &keys) {
   mini_->CreateTable(table, family, keys);
 }
-void TestUtil::TablePut(const std::string &table, const std::string &row, const std::string &family,
-                        const std::string &column, const std::string &value) {
-  mini_->TablePut(table, row, family, column, value);
-}
 
 void TestUtil::StartStandAloneInstance() {
   auto p = temp_dir_.path().string();

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f8f36e/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
index b5714ee..5729674 100644
--- a/hbase-native-client/test-util/test-util.h
+++ b/hbase-native-client/test-util/test-util.h
@@ -61,8 +61,6 @@ class TestUtil {
   void CreateTable(const std::string &table, const std::string &family);
   void CreateTable(const std::string &table, const std::string &family,
                    const std::vector<std::string> &keys);
-  void TablePut(const std::string &table, const std::string &row, const std::string &family,
-                const std::string &column, const std::string &value);
 
   void StartStandAloneInstance();
   void StopStandAloneInstance();