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

[hbase] 76/133: HBASE-15894 [C++] Put and Mutation objects

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

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

commit 962995e907384acf1b0661e7ded3f6cfc9f13af0
Author: Enis Soztutar <en...@apache.org>
AuthorDate: Wed Mar 29 16:34:20 2017 -0700

    HBASE-15894 [C++] Put and Mutation objects
---
 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(-)

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",
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());
+}
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 */
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
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
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_; }
 
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>();
 };
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
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
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
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
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) {
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) {
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;
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);
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
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,
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"
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",],)
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"));
+}
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 */
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 */