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 2016/12/07 22:52:29 UTC

hbase git commit: HBASE-15902 Scan Object

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14850 99ccddff5 -> 7e59ff620


HBASE-15902 Scan Object

Signed-off-by: Enis Soztutar <en...@apache.org>


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

Branch: refs/heads/HBASE-14850
Commit: 7e59ff620f482b38a6817262bb0a56013b4c28f1
Parents: 99ccddf
Author: Sudeep Sunthankar <su...@hashmapinc.com>
Authored: Sat Dec 3 11:35:11 2016 +1100
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Dec 7 14:52:05 2016 -0800

----------------------------------------------------------------------
 hbase-native-client/core/BUCK         |   8 +
 hbase-native-client/core/get-test.cc  | 253 ++++++++++++++++++-----------
 hbase-native-client/core/get.cc       | 109 +++++--------
 hbase-native-client/core/get.h        |  40 ++---
 hbase-native-client/core/scan-test.cc | 233 ++++++++++++++++++++++++++
 hbase-native-client/core/scan.cc      | 239 +++++++++++++++++++++++++++
 hbase-native-client/core/scan.h       | 253 +++++++++++++++++++++++++++++
 7 files changed, 953 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/hbase-native-client/core/BUCK
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 9dbc82e..fa01eb3 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -31,6 +31,7 @@ cxx_library(
         "time_range.h",
         "configuration.h",
         "hbase_configuration_loader.h",
+        "scan.h",
     ],
     srcs=[
         "cell.cc",
@@ -41,6 +42,7 @@ cxx_library(
         "time_range.cc",
         "configuration.cc",
         "hbase_configuration_loader.cc",
+        "scan.cc",
     ],
     deps=[
         "//connection:connection",
@@ -85,6 +87,12 @@ cxx_test(name="hbase_configuration-test",
          ],
          deps=[":core", ],
          run_test_separately=True, )
+cxx_test(name="scan-test",
+         srcs=[
+             "scan-test.cc",
+         ],
+         deps=[":core", ],
+         run_test_separately=True, )
 cxx_binary(name="simple-client",
            srcs=["simple-client.cc", ],
            deps=[":core", "//connection:connection"], )

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/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 8ed5f2f..9a09a23 100644
--- a/hbase-native-client/core/get-test.cc
+++ b/hbase-native-client/core/get-test.cc
@@ -21,129 +21,200 @@
 
 #include <gtest/gtest.h>
 #include <glog/logging.h>
-
 using namespace hbase;
 const int NUMBER_OF_GETS = 5;
 
-TEST (Get, SingleGet) {
-
-  std::string row_str = "row-test";
-  ASSERT_NO_THROW(Get tmp = Get(row_str));
-  Get get = Get(row_str);
-
-  get.SetCacheBlocks(true);
-  get.SetConsistency(hbase::pb::Consistency::STRONG);
-  get.SetMaxResultsPerColumnFamily(1);
-
-  ASSERT_THROW(get.SetMaxVersions(0), std::runtime_error);
-  ASSERT_NO_THROW(get.SetMaxVersions(-10));
-  ASSERT_THROW(get.SetMaxVersions(std::numeric_limits<unsigned int>::max() + 1),
-               std::runtime_error);
-
-  ASSERT_THROW(get.SetTimeRange(-100, 2000), std::runtime_error);
-  ASSERT_THROW(get.SetTimeRange(100, -2000), std::runtime_error);
-  ASSERT_THROW(get.SetTimeRange(1000, 200), std::runtime_error);
-
-  ASSERT_NO_THROW(get.SetMaxVersions());
-  ASSERT_NO_THROW(get.SetMaxVersions(2));
-  ASSERT_NO_THROW(get.SetTimeRange(0, std::numeric_limits<long>::max()));
-
-  EXPECT_EQ(true, get.CacheBlocks());
-  EXPECT_EQ(hbase::pb::Consistency::STRONG, get.Consistency());
-  EXPECT_EQ(1, get.MaxResultsPerColumnFamily());
-  EXPECT_EQ(2, get.MaxVersions());
-
-  TimeRange tr = get.Timerange();
-  EXPECT_EQ(0, tr.MinTimeStamp());
-  EXPECT_EQ(std::numeric_limits<long>::max(), tr.MaxTimeStamp());
-
-  EXPECT_EQ("row-test", get.Row());
-
+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_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());
+  get.AddFamily("family-2");
+  EXPECT_EQ(2, get.Family().size());
+  get.AddFamily("family-3");
+  EXPECT_EQ(3, get.Family().size());
+  int i = 1;
+  for (const auto &family : get.Family()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+    i += 1;
+  }
 
   get.AddColumn("family-1", "column-1");
   get.AddColumn("family-1", "column-2");
   get.AddColumn("family-1", "");
   get.AddColumn("family-1", "column-3");
-  get.AddFamily("family-1");
-  get.AddFamily("family-2");
-  get.AddFamily("family-3");
-
+  get.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, get.Family().size());
+  auto it = get.Family().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, get.Family().end());
 }
 
-TEST (Get, MultiGet) {
-
-  std::vector<Get *> gets;
-  for (int i = 0; i < NUMBER_OF_GETS; i++) {
-    std::string row_str = "row-test";
-    row_str += std::to_string(i);
-    ASSERT_NO_THROW(Get tmp = Get(row_str));
-    Get *get = new Get(row_str);
-
-    get->SetCacheBlocks(true);
-    get->SetConsistency(hbase::pb::Consistency::STRONG);
-    get->SetMaxResultsPerColumnFamily(1);
-
-    ASSERT_THROW(get->SetMaxVersions(0), std::runtime_error);
-    ASSERT_NO_THROW(get->SetMaxVersions(-10));
-    ASSERT_THROW(
-        get->SetMaxVersions(std::numeric_limits<unsigned int>::max() + 1),
-        std::runtime_error);
+void CheckFamiliesAfterCopy(Get &get) {
+  EXPECT_EQ(true, get.HasFamilies());
+  EXPECT_EQ(3, get.Family().size());
+  int i = 1;
+  for (const auto &family : get.Family()) {
+    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();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, get.Family().end());
+}
 
-    ASSERT_THROW(get->SetTimeRange(-100, 2000), std::runtime_error);
-    ASSERT_THROW(get->SetTimeRange(100, -2000), std::runtime_error);
-    ASSERT_THROW(get->SetTimeRange(1000, 200), std::runtime_error);
+void GetMethods(Get &get, const std::string &row) {
+  EXPECT_EQ(row, get.Row());
 
-    get->SetMaxVersions();
-    get->SetMaxVersions(2);
-    get->SetTimeRange(0, std::numeric_limits<long>::max());
+  CheckFamilies(get);
+  EXPECT_EQ(true, get.CacheBlocks());
+  get.SetCacheBlocks(false);
+  EXPECT_EQ(false, get.CacheBlocks());
 
-    EXPECT_EQ(true, get->CacheBlocks());
-    EXPECT_EQ(hbase::pb::Consistency::STRONG, get->Consistency());
-    EXPECT_EQ(1, get->MaxResultsPerColumnFamily());
-    EXPECT_EQ(2, get->MaxVersions());
+  EXPECT_EQ(hbase::pb::Consistency::STRONG, get.Consistency());
+  get.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, get.Consistency());
 
-    TimeRange tr = get->Timerange();
-    EXPECT_EQ(0, tr.MinTimeStamp());
-    EXPECT_EQ(std::numeric_limits<long>::max(), tr.MaxTimeStamp());
+  EXPECT_EQ(1, get.MaxVersions());
+  get.SetMaxVersions(2);
+  EXPECT_EQ(2, get.MaxVersions());
+  get.SetMaxVersions();
+  EXPECT_EQ(1, get.MaxVersions());
+
+  // Test initial values
+  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(std::numeric_limits<long>::max(), get.Timerange().MaxTimeStamp());
+
+  // Set & Test new values using TimeRange and TimeStamp
+  get.SetTimeRange(1000, 2000);
+  EXPECT_EQ(1000, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(2000, get.Timerange().MaxTimeStamp());
+  get.SetTimeStamp(0);
+  EXPECT_EQ(0, get.Timerange().MinTimeStamp());
+  EXPECT_EQ(1, get.Timerange().MaxTimeStamp());
+
+  // Test some exceptions
+  ASSERT_THROW(get.SetTimeRange(-1000, 2000), std::runtime_error);
+  ASSERT_THROW(get.SetTimeRange(1000, -2000), std::runtime_error);
+  ASSERT_THROW(get.SetTimeRange(1000, 200), std::runtime_error);
+  ASSERT_THROW(get.SetTimeStamp(std::numeric_limits<long>::max()), std::runtime_error);
 
-    EXPECT_EQ(false, get->HasFamilies());
+  //Test some exceptions
+  ASSERT_THROW(get.SetMaxVersions(0), std::runtime_error);
+  ASSERT_THROW(get.SetMaxVersions(std::numeric_limits<unsigned int>::max() + 1),
+               std::runtime_error);
+}
 
-    get->AddFamily("family-1");
-    EXPECT_EQ(true, get->HasFamilies());
+TEST (Get, SingleGet) {
+  std::string row_str = "row-test";
+  Get get(row_str);
+  GetMethods(get, row_str);
+
+  Get get_tmp(row_str);
+  Get getcp(get_tmp);
+  GetMethods(getcp, row_str);
+
+  Get geteq("test");
+  geteq = get_tmp;
+  GetMethods(geteq, row_str);
+
+  // Adding the below tests as there were some concerns raised that the same vector of qualifiers in FamilyMap is being shared between copied objects
+  // Verify the source object's family map size before using it to copy.
+  EXPECT_EQ(3, get.Family().size());
+
+  Get getcp_fam(get);
+  // address of family maps should be different.
+  EXPECT_NE(&(get.Family()), &(getcp_fam.Family()));
+
+  // 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());
+  // Verify the source object's family elements
+  auto it = get.Family().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_STREQ("family-4", it->first.c_str());
+  EXPECT_EQ(2, it->second.size());
+  EXPECT_STREQ("column-A", it->second[0].c_str());
+  EXPECT_STREQ("column-B", it->second[1].c_str());
+  ++it;
+  EXPECT_EQ(it, get.Family().end());
+
+  //Verifying the copied object's families. It will remain unchanged and below tests should pass
+  CheckFamiliesAfterCopy(getcp_fam);
+}
 
-    get->AddColumn("family-1", "column-1");
-    get->AddColumn("family-1", "column-2");
-    get->AddColumn("family-1", "");
-    get->AddColumn("family-1", "column-3");
-    get->AddFamily("family-1");
-    get->AddFamily("family-2");
-    get->AddFamily("family-3");
+TEST (Get, MultiGet) {
+  std::vector<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);
 
+    GetMethods(*get, row_str);
     gets.push_back(get);
   }
   EXPECT_EQ(NUMBER_OF_GETS, gets.size());
 
-  int i = 0;
-  for (const auto &get : gets) {
-    std::string row_str = "row-test";
-    row_str += std::to_string(i);
-    EXPECT_EQ(row_str, get->Row());
-    i++;
-  }
-
   for (const auto &get : gets) {
     delete get;
   }
   gets.clear();
-
 }
 
 TEST (Get, Exception) {
-
   std::string row(std::numeric_limits<short>::max() + 1, 'X');
   ASSERT_THROW(Get tmp = Get(row), std::runtime_error);
   ASSERT_THROW(Get tmp = Get(""), std::runtime_error);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/hbase-native-client/core/get.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.cc b/hbase-native-client/core/get.cc
index b0d91ec..b412a4b 100644
--- a/hbase-native-client/core/get.cc
+++ b/hbase-native-client/core/get.cc
@@ -25,42 +25,31 @@
 namespace hbase {
 
 Get::~Get() {
-
 }
 
 Get::Get(const std::string &row)
-    : row_(row),
-      max_versions_(1),
-      cache_blocks_(true),
-      store_limit_(-1),
-      store_offset_(0),
-      check_existence_only_(false),
-      consistency_(hbase::pb::Consistency::STRONG),
-      tr_(TimeRange()) {
-  Get::CheckRow(&row_);
-  family_map_.clear();
-}
-
-Get::Get(const Get &cget) {
-  this->row_ = cget.row_;
-  this->max_versions_ = cget.max_versions_;
-  this->cache_blocks_ = cget.cache_blocks_;
-  this->store_limit_ = cget.store_limit_;
-  this->store_offset_ = cget.store_offset_;
-  this->check_existence_only_ = cget.check_existence_only_;
-  this->consistency_ = cget.consistency_;
-  this->tr_ = cget.tr_;
-}
-
-Get& Get::operator=(const Get &cget) {
-  this->row_ = cget.row_;
-  this->max_versions_ = cget.max_versions_;
-  this->cache_blocks_ = cget.cache_blocks_;
-  this->store_limit_ = cget.store_limit_;
-  this->store_offset_ = cget.store_offset_;
-  this->check_existence_only_ = cget.check_existence_only_;
-  this->consistency_ = cget.consistency_;
-  this->tr_ = cget.tr_;
+    : row_(row) {
+  CheckRow(row_);
+}
+
+Get::Get(const Get &get) {
+  row_ = get.row_;
+  max_versions_ = get.max_versions_;
+  cache_blocks_ = get.cache_blocks_;
+  check_existence_only_ = get.check_existence_only_;
+  consistency_ = get.consistency_;
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
+}
+
+Get& Get::operator=(const Get &get) {
+  row_ = get.row_;
+  max_versions_ = get.max_versions_;
+  cache_blocks_ = get.cache_blocks_;
+  check_existence_only_ = get.check_existence_only_;
+  consistency_ = get.consistency_;
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.family_map_.begin(), get.family_map_.end());
   return *this;
 }
 
@@ -80,15 +69,14 @@ Get& Get::AddFamily(const std::string &family) {
 }
 
 Get& Get::AddColumn(const std::string &family, const std::string &qualifier) {
-  const auto &it = std::find(this->family_map_[family].begin(),
-                             this->family_map_[family].end(), qualifier);
+  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
 
   /**
    * Check if any qualifiers are already present or not.
    * Add only if qualifiers for a given family are not present
    */
-  if (it == this->family_map_[family].end()) {
-    this->family_map_[family].push_back(qualifier);
+  if (it == family_map_[family].end()) {
+    family_map_[family].push_back(qualifier);
   }
   return *this;
 }
@@ -98,76 +86,67 @@ const std::string& Get::Row() const {
 }
 
 hbase::pb::Consistency Get::Consistency() const {
-  return this->consistency_;
+  return consistency_;
 }
 
 Get &Get::SetConsistency(hbase::pb::Consistency consistency) {
-  this->consistency_ = consistency;
+  consistency_ = consistency;
   return *this;
 }
 
-bool Get::HasFamilies() {
-  return !this->family_map_.empty();
+bool Get::HasFamilies() const {
+  return !family_map_.empty();
 }
 
-const FAMILY_MAP &Get::FamilyMap() const {
-  return this->family_map_;
+const FamilyMap &Get::Family() const {
+  return family_map_;
 }
 
 int Get::MaxVersions() const {
-  return this->max_versions_;
+  return max_versions_;
 }
 
 Get& Get::SetMaxVersions(uint32_t max_versions) {
   if (0 == max_versions)
     throw std::runtime_error("max_versions must be positive");
 
-  this->max_versions_ = max_versions;
+  max_versions_ = max_versions;
   return *this;
 }
 
 bool Get::CacheBlocks() const {
-  return this->cache_blocks_;
+  return cache_blocks_;
 }
 
 Get & Get::SetCacheBlocks(bool cache_blocks) {
-  this->cache_blocks_ = cache_blocks;
-  return *this;
-}
-
-int Get::MaxResultsPerColumnFamily() const {
-  return this->store_limit_;
-}
-
-Get& Get::SetMaxResultsPerColumnFamily(int store_limit) {
-  this->store_limit_ = store_limit;
+  cache_blocks_ = cache_blocks;
   return *this;
 }
 
 Get& Get::SetTimeRange(long min_timestamp, long max_timestamp) {
-  this->tr_ = TimeRange(min_timestamp, max_timestamp);
+  tr_.reset(new TimeRange(min_timestamp, max_timestamp));
   return *this;
 }
 
 Get& Get::SetTimeStamp(long timestamp) {
-  this->tr_ = TimeRange(timestamp, timestamp + 1);
+  tr_.reset(new TimeRange(timestamp, timestamp + 1));
   return *this;
 }
 
 const TimeRange& Get::Timerange() const {
-  return this->tr_;
+  return *tr_;
 }
 
-void Get::CheckRow(const std::string *row) {
-  int MAX_ROW_LENGTH = std::numeric_limits<short>::max();
-  int row_length = row->size();
+void Get::CheckRow(const std::string &row) {
+  const int kMaxRowLength = std::numeric_limits<short>::max();
+  int row_length = row.size();
   if (0 == row_length) {
     throw std::runtime_error("Row length can't be 0");
   }
-  if (row_length > MAX_ROW_LENGTH) {
+  if (row_length > kMaxRowLength) {
     throw std::runtime_error(
-        "Length of " + *row + " is greater than max row size: "
-            + std::to_string(MAX_ROW_LENGTH));
+        "Length of " + row + " is greater than max row size: " + std::to_string(kMaxRowLength));
   }
 }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/hbase-native-client/core/get.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/get.h b/hbase-native-client/core/get.h
index 28141f2..9704aa1 100644
--- a/hbase-native-client/core/get.h
+++ b/hbase-native-client/core/get.h
@@ -20,6 +20,7 @@
 #pragma once
 
 #include <map>
+#include <memory>
 #include <string>
 #include <vector>
 #include <cstdint>
@@ -31,7 +32,7 @@ namespace hbase {
 /**
  * @brief Map consisting of column families and qualifiers to be used for Get operation
  */
-using FAMILY_MAP = std::map<std::string, std::vector<std::string>>;
+using FamilyMap = std::map<std::string, std::vector<std::string>>;
 
 class Get {
 
@@ -69,20 +70,9 @@ class Get {
   Get& SetCacheBlocks(bool cache_blocks);
 
   /**
-   * @brief Method for retrieving the get's maximum number of values to return per Column Family
+   * @brief Returns the Get family map (FamilyMap) for this Get operation. Used for constructing Scan object with an already constructed Get
    */
-  int MaxResultsPerColumnFamily() const;
-
-  /**
-   * @brief Set the maximum number of values to return per row per Column Family
-   * @param the store_limit to be set
-   */
-  Get& SetMaxResultsPerColumnFamily(int store_limit);
-
-  /**
-   * @brief Returns the Get family map (FAMILY_MAP) for this Get operation.
-   */
-  const FAMILY_MAP &FamilyMap() const;
+  const FamilyMap &Family() const;
 
   /**
    * @brief Returns the timerange for this Get
@@ -121,9 +111,9 @@ class Get {
   const std::string& Row() const;
 
   /**
-   * @brief Returns true if family map (FAMILY_MAP) is non empty false otherwise
+   * @brief Returns true if family map (FamilyMap) is non empty false otherwise
    */
-  bool HasFamilies();
+  bool HasFamilies() const;
 
   /**
    * @brief Returns the consistency level for this Get operation
@@ -137,21 +127,19 @@ class Get {
   Get& SetConsistency(hbase::pb::Consistency consistency);
 
  private:
-  std::string row_;
-  uint32_t max_versions_;
-  bool cache_blocks_;
-  int store_limit_;
-  int store_offset_;
-  bool check_existence_only_;
-  FAMILY_MAP family_map_;
-  hbase::pb::Consistency consistency_;
-  TimeRange tr_;
+  std::string row_ = "";
+  uint32_t max_versions_ = 1;
+  bool cache_blocks_ = true;
+  bool check_existence_only_ = false;
+  FamilyMap family_map_;
+  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
+  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
 
   /**
    * @brief Checks if the row for this Get operation is proper or not
    * @param row Row to check
    * @throws std::runtime_error if row is empty or greater than MAX_ROW_LENGTH(i.e. std::numeric_limits<short>::max())
    */
-  void CheckRow(const std::string *row);
+  void CheckRow(const std::string &row);
 };
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/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
new file mode 100644
index 0000000..a114502
--- /dev/null
+++ b/hbase-native-client/core/scan-test.cc
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#include "core/scan.h"
+
+#include <limits>
+
+#include <gtest/gtest.h>
+
+using namespace hbase;
+
+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_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());
+  scan.AddFamily("family-2");
+  EXPECT_EQ(2, scan.Family().size());
+  scan.AddFamily("family-3");
+  EXPECT_EQ(3, scan.Family().size());
+  int i = 1;
+  for (const auto &family : scan.Family()) {
+    std::string family_name = "family-" + std::to_string(i);
+    EXPECT_STREQ(family_name.c_str(), family.first.c_str());
+    EXPECT_EQ(0, family.second.size());
+    i += 1;
+  }
+
+  scan.AddColumn("family-1", "column-1");
+  scan.AddColumn("family-1", "column-2");
+  scan.AddColumn("family-1", "");
+  scan.AddColumn("family-1", "column-3");
+  scan.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, scan.Family().size());
+  auto it = scan.Family().begin();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, scan.Family().end());
+}
+
+void CheckFamiliesAfterCopy(Scan &scan) {
+  EXPECT_EQ(true, scan.HasFamilies());
+  EXPECT_EQ(3, scan.Family().size());
+  int i = 1;
+  for (const auto &family : scan.Family()) {
+    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();
+  EXPECT_STREQ("family-1", it->first.c_str());
+  EXPECT_EQ(4, it->second.size());
+  EXPECT_STREQ("column-1", it->second[0].c_str());
+  EXPECT_STREQ("column-2", it->second[1].c_str());
+  EXPECT_STREQ("", it->second[2].c_str());
+  EXPECT_STREQ("column-3", it->second[3].c_str());
+  ++it;
+  EXPECT_STREQ("family-2", it->first.c_str());
+  EXPECT_EQ(1, it->second.size());
+  EXPECT_STREQ("column-X", it->second[0].c_str());
+  ++it;
+  EXPECT_STREQ("family-3", it->first.c_str());
+  EXPECT_EQ(0, it->second.size());
+  ++it;
+  EXPECT_EQ(it, scan.Family().end());
+}
+
+void ScanMethods(Scan &scan) {
+  scan.SetReversed(true);
+  EXPECT_EQ(true, scan.IsReversed());
+  scan.SetReversed(false);
+  EXPECT_EQ(false, scan.IsReversed());
+
+  std::string start_row("start-row");
+  std::string stop_row("stop-row");
+  scan.SetStartRow(start_row);
+  EXPECT_EQ(start_row, scan.StartRow());
+
+  scan.SetStopRow(stop_row);
+  EXPECT_EQ(stop_row, scan.StopRow());
+
+  scan.SetSmall(true);
+  EXPECT_EQ(true, scan.IsSmall());
+  scan.SetSmall(false);
+  EXPECT_EQ(false, scan.IsSmall());
+
+  scan.SetCaching(3);
+  EXPECT_EQ(3, scan.Caching());
+
+  scan.SetConsistency(hbase::pb::Consistency::STRONG);
+  EXPECT_EQ(hbase::pb::Consistency::STRONG, scan.Consistency());
+  scan.SetConsistency(hbase::pb::Consistency::TIMELINE);
+  EXPECT_EQ(hbase::pb::Consistency::TIMELINE, scan.Consistency());
+
+  scan.SetCacheBlocks(true);
+  EXPECT_EQ(true, scan.CacheBlocks());
+  scan.SetCacheBlocks(false);
+  EXPECT_EQ(false, scan.CacheBlocks());
+
+  scan.SetAllowPartialResults(true);
+  EXPECT_EQ(true, scan.AllowPartialResults());
+  scan.SetAllowPartialResults(false);
+  EXPECT_EQ(false, scan.AllowPartialResults());
+
+  scan.SetLoadColumnFamiliesOnDemand(true);
+  EXPECT_EQ(true, scan.LoadColumnFamiliesOnDemand());
+  scan.SetLoadColumnFamiliesOnDemand(false);
+  EXPECT_EQ(false, scan.LoadColumnFamiliesOnDemand());
+
+  scan.SetMaxVersions();
+  EXPECT_EQ(1, scan.MaxVersions());
+  scan.SetMaxVersions(20);
+  EXPECT_EQ(20, scan.MaxVersions());
+
+  scan.SetMaxResultSize(1024);
+  EXPECT_EQ(1024, scan.MaxResultSize());
+
+  // Test initial values
+  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(std::numeric_limits<long>::max(), scan.Timerange().MaxTimeStamp());
+
+  // Set & Test new values using TimeRange and TimeStamp
+  scan.SetTimeRange(1000, 2000);
+  EXPECT_EQ(1000, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(2000, scan.Timerange().MaxTimeStamp());
+  scan.SetTimeStamp(0);
+  EXPECT_EQ(0, scan.Timerange().MinTimeStamp());
+  EXPECT_EQ(1, scan.Timerange().MaxTimeStamp());
+
+  // Test some exceptions
+  ASSERT_THROW(scan.SetTimeRange(-1000, 2000), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeRange(1000, -2000), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeRange(1000, 200), std::runtime_error);
+  ASSERT_THROW(scan.SetTimeStamp(std::numeric_limits<long>::max()), std::runtime_error);
+}
+
+TEST (Scan, Object) {
+  Scan scan;
+  ScanMethods(scan);
+  CheckFamilies(scan);
+
+  // Resetting TimeRange values so that the copy construction and assignment operator tests pass.
+  scan.SetTimeRange(0, std::numeric_limits<long>::max());
+  Scan scancp(scan);
+  ScanMethods(scancp);
+  CheckFamiliesAfterCopy(scancp);
+
+  Scan scaneq;
+  scaneq = scan;
+  ScanMethods(scaneq);
+  CheckFamiliesAfterCopy(scaneq);
+}
+
+TEST (Scan, WithStartRow) {
+  Scan("row-test");
+  Scan scan("row-test");
+  ScanMethods(scan);
+  CheckFamilies(scan);
+}
+
+TEST (Scan, WithStartAndStopRow) {
+  Scan("start-row", "stop-row");
+  Scan scan("start-row", "stop-row");
+  ScanMethods(scan);
+  CheckFamilies(scan);
+}
+
+TEST (Scan,FromGet) {
+
+  std::string row_str = "row-test";
+  Get get = Get(row_str);
+
+  get.SetCacheBlocks(true);
+  get.SetMaxVersions(5);
+  get.AddFamily("family-1");
+  get.AddFamily("family-1");
+  get.AddFamily("family-2");
+  get.AddFamily("family-3");
+  get.AddColumn("family-1", "column-1");
+  get.AddColumn("family-1", "column-2");
+  get.AddColumn("family-1", "");
+  get.AddColumn("family-1", "column-3");
+  get.AddColumn("family-2", "column-X");
+
+  EXPECT_EQ(3, get.Family().size());
+
+  Scan scan(get);
+  ScanMethods(scan);
+  CheckFamiliesAfterCopy(scan);
+}
+
+TEST (Scan, Exception) {
+  std::string row(std::numeric_limits<short>::max() + 1, 'X');
+  ASSERT_THROW(Scan tmp(row), std::runtime_error);
+  ASSERT_THROW(Scan tmp(""), std::runtime_error);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/hbase-native-client/core/scan.cc
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.cc b/hbase-native-client/core/scan.cc
new file mode 100644
index 0000000..318cdb7
--- /dev/null
+++ b/hbase-native-client/core/scan.cc
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#include "core/scan.h"
+
+#include <algorithm>
+#include <iterator>
+#include <limits>
+#include <stdexcept>
+
+namespace hbase {
+
+Scan::Scan() {
+}
+
+Scan::~Scan() {
+}
+
+Scan::Scan(const std::string &start_row)
+    : start_row_(start_row) {
+  CheckRow(start_row_);
+}
+
+Scan::Scan(const std::string &start_row, const std::string &stop_row)
+    : start_row_(start_row),
+      stop_row_(stop_row) {
+  CheckRow(start_row_);
+  CheckRow(stop_row_);
+}
+
+Scan::Scan(const Scan &scan) {
+  start_row_ = scan.start_row_;
+  stop_row_ = scan.stop_row_;
+  max_versions_ = scan.max_versions_;
+  caching_ = scan.caching_;
+  max_result_size_ = scan.max_result_size_;
+  cache_blocks_ = scan.cache_blocks_;
+  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
+  reversed_ = scan.reversed_;
+  small_ = scan.small_;
+  allow_partial_results_ = scan.allow_partial_results_;
+  consistency_ = scan.consistency_;
+  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
+  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
+}
+
+Scan& Scan::operator=(const Scan &scan) {
+  start_row_ = scan.start_row_;
+  stop_row_ = scan.stop_row_;
+  max_versions_ = scan.max_versions_;
+  caching_ = scan.caching_;
+  max_result_size_ = scan.max_result_size_;
+  cache_blocks_ = scan.cache_blocks_;
+  load_column_families_on_demand_ = scan.load_column_families_on_demand_;
+  reversed_ = scan.reversed_;
+  small_ = scan.small_;
+  allow_partial_results_ = scan.allow_partial_results_;
+  consistency_ = scan.consistency_;
+  tr_.reset(new TimeRange(scan.tr_->MinTimeStamp(), scan.tr_->MaxTimeStamp()));
+  family_map_.insert(scan.family_map_.begin(), scan.family_map_.end());
+  return *this;
+}
+
+Scan::Scan(const Get &get) {
+  cache_blocks_ = get.CacheBlocks();
+  max_versions_ = get.MaxVersions();
+  tr_.reset(new TimeRange(get.Timerange().MinTimeStamp(), get.Timerange().MaxTimeStamp()));
+  family_map_.insert(get.Family().begin(), get.Family().end());
+}
+
+Scan& Scan::AddFamily(const std::string &family) {
+  const auto &it = family_map_.find(family);
+  /**
+   * Check if any qualifiers are already present or not.
+   * Remove all existing qualifiers if the given family is already present in the map
+   */
+  if (family_map_.end() != it) {
+    it->second.clear();
+  } else {
+    family_map_[family];
+  }
+  return *this;
+}
+
+Scan& Scan::AddColumn(const std::string &family, const std::string &qualifier) {
+  const auto &it = std::find(family_map_[family].begin(), family_map_[family].end(), qualifier);
+  /**
+   * Check if any qualifiers are already present or not.
+   * Add only if qualifiers for a given family are not present
+   */
+  if (it == family_map_[family].end()) {
+    family_map_[family].push_back(qualifier);
+  }
+  return *this;
+}
+
+void Scan::SetReversed(bool reversed) {
+  reversed_ = reversed;
+}
+
+bool Scan::IsReversed() const {
+  return reversed_;
+}
+
+void Scan::SetStartRow(std::string &start_row) {
+  CheckRow(start_row);
+  start_row_ = start_row;
+}
+
+const std::string& Scan::StartRow() const {
+  return start_row_;
+}
+
+void Scan::SetStopRow(std::string &stop_row) {
+  CheckRow(stop_row);
+  stop_row_ = stop_row;
+}
+
+const std::string& Scan::StopRow() const {
+  return stop_row_;
+}
+
+void Scan::SetSmall(bool small) {
+  small_ = small;
+}
+
+bool Scan::IsSmall() const {
+  return small_;
+}
+
+void Scan::SetCaching(int caching) {
+  caching_ = caching;
+}
+
+int Scan::Caching() const {
+  return caching_;
+}
+
+Scan& Scan::SetConsistency(const hbase::pb::Consistency consistency) {
+  consistency_ = consistency;
+  return *this;
+}
+
+hbase::pb::Consistency Scan::Consistency() const {
+  return consistency_;
+}
+
+void Scan::SetCacheBlocks(bool cache_blocks) {
+  cache_blocks_ = cache_blocks;
+}
+
+bool Scan::CacheBlocks() const {
+  return cache_blocks_;
+}
+
+void Scan::SetAllowPartialResults(bool allow_partial_results) {
+  allow_partial_results_ = allow_partial_results;
+}
+
+bool Scan::AllowPartialResults() const {
+  return allow_partial_results_;
+}
+
+void Scan::SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand) {
+  load_column_families_on_demand_ = load_column_families_on_demand;
+}
+
+bool Scan::LoadColumnFamiliesOnDemand() const {
+  return load_column_families_on_demand_;
+}
+
+Scan& Scan::SetMaxVersions(uint32_t max_versions) {
+  max_versions_ = max_versions;
+  return *this;
+}
+
+int Scan::MaxVersions() const {
+  return max_versions_;
+}
+
+void Scan::SetMaxResultSize(long max_result_size) {
+  max_result_size_ = max_result_size;
+}
+
+long Scan::MaxResultSize() const {
+  return max_result_size_;
+}
+
+Scan &Scan::SetTimeRange(long min_stamp, long max_stamp) {
+  tr_.reset(new TimeRange(min_stamp, max_stamp));
+  return *this;
+}
+
+Scan& Scan::SetTimeStamp(long timestamp) {
+  tr_.reset(new TimeRange(timestamp, timestamp + 1));
+  return *this;
+}
+
+const TimeRange& Scan::Timerange() const {
+  return *tr_;
+}
+
+void Scan::CheckRow(const std::string &row) {
+  const int kMaxRowLength = std::numeric_limits<short>::max();
+  int row_length = row.size();
+  if (0 == row_length) {
+    throw std::runtime_error("Row length can't be 0");
+  }
+  if (row_length > kMaxRowLength) {
+    throw std::runtime_error(
+        "Length of " + row + " is greater than max row size: " + std::to_string(kMaxRowLength));
+  }
+}
+
+bool Scan::HasFamilies() const {
+  return !family_map_.empty();
+}
+
+const FamilyMap &Scan::Family() const {
+  return family_map_;
+}
+}
+/* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/7e59ff62/hbase-native-client/core/scan.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/core/scan.h b/hbase-native-client/core/scan.h
new file mode 100644
index 0000000..bd655ce
--- /dev/null
+++ b/hbase-native-client/core/scan.h
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "core/get.h"
+#include "core/time_range.h"
+#include "if/Client.pb.h"
+
+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:
+  /**
+   * @brief Constructors. Create a Scan operation across all rows.
+   */
+  Scan();
+  Scan(const Scan &scan);
+  Scan& operator=(const Scan &scan);
+
+  ~Scan();
+
+  /**
+   * @brief Create a Scan operation starting at the specified row. If the specified row does not exist,
+   * the Scanner will start from the next closest row after the specified row.
+   * @param start_row - row to start scanner at or after
+   */
+  Scan(const std::string &start_row);
+
+  /**
+   * @brief Create a Scan operation for the range of rows specified.
+   * @param start_row - row to start scanner at or after (inclusive).
+   * @param stop_row - row to stop scanner before (exclusive).
+   */
+  Scan(const std::string &start_row, const std::string &stop_row);
+
+  /**
+   * @brief Builds a scan object with the same specs as get.
+   * @param get - get to model scan after
+   */
+  Scan(const Get &get);
+
+  /**
+   * @brief Get all columns from the specified family.Overrides previous calls to AddColumn for this family.
+   * @param family - family name
+   */
+  Scan& AddFamily(const std::string &family);
+
+  /**
+   * @brief Get the column from the specified family with the specified qualifier.Overrides previous calls to AddFamily for this family.
+   * @param family - family name.
+   * @param qualifier - column qualifier.
+   */
+  Scan& AddColumn(const std::string &family, const std::string &qualifier);
+
+  /**
+   * @brief Set whether this scan is a reversed one. This is false by default which means forward(normal) scan.
+   * @param reversed - if true, scan will be backward order
+   */
+  void SetReversed(bool reversed);
+
+  /**
+   * @brief Get whether this scan is a reversed one. Returns  true if backward scan, false if forward(default) scan
+   */
+  bool IsReversed() const;
+
+  /**
+   * @brief Set the start row of the scan.If the specified row does not exist, the Scanner will start from the next closest row after the specified row.
+   * @param start_row - row to start scanner at or after
+   * @throws std::runtime_error if start_row length is 0 or greater than MAX_ROW_LENGTH
+   */
+  void SetStartRow(std::string &start_row);
+
+  /**
+   * @brief returns start_row of the Scan.
+   */
+  const std::string& StartRow() const;
+
+  /**
+   * @brief Set the stop row of the scan. The scan will include rows that are lexicographically less than the provided stop_row.
+   * @param stop_row - row to end at (exclusive)
+   * @throws std::runtime_error if stop_row length is 0 or greater than MAX_ROW_LENGTH
+   */
+  void SetStopRow(std::string &stop_row);
+
+  /**
+   * @brief returns stop_row of the Scan.
+   */
+  const std::string& StopRow() const;
+
+  /**
+   * @brief Set whether this scan is a small scan.
+   */
+  void SetSmall(bool small);
+
+  /**
+   * @brief Returns if the scan is a small scan.
+   */
+  bool IsSmall() const;
+
+  /**
+   * @brief Set the number of rows for caching that will be passed to scanners. Higher caching values will enable faster scanners but will use more memory.
+   * @param caching - the number of rows for caching.
+   */
+  void SetCaching(int caching);
+
+  /**
+   * @brief caching the number of rows fetched when calling next on a scanner.
+   */
+  int Caching() const;
+
+  /**
+   * @brief Sets the consistency level for this operation.
+   * @param consistency - the consistency level
+   */
+  Scan& SetConsistency(const hbase::pb::Consistency consistency);
+
+  /**
+   * @brief Returns the consistency level for this operation.
+   */
+  hbase::pb::Consistency Consistency() const;
+
+  /**
+   * @brief Set whether blocks should be cached for this Scan.This is true by default. When true, default settings of the table and family are used (this will never override caching blocks if the block cache is disabled for that family or entirely).
+   * @param cache_blocks - if false, default settings are overridden and blocks will not be cached
+   */
+  void SetCacheBlocks(bool cache_blocks);
+
+  /**
+   * @brief Get whether blocks should be cached for this Scan.
+   */
+  bool CacheBlocks() const;
+
+  /**
+   * @brief Setting whether the caller wants to see the partial results that may be returned from the server. By default this value is false and the complete results will be assembled client side before being delivered to the caller.
+   * @param allow_partial_results - if true partial results will be returned.
+   */
+  void SetAllowPartialResults(bool allow_partial_results);
+
+  /**
+   * @brief true when the constructor of this scan understands that the results they will see may only represent a partial portion of a row. The entire row would be retrieved by subsequent calls to ResultScanner.next()
+   */
+  bool AllowPartialResults() const;
+
+  /**
+   * @brief Set the value indicating whether loading CFs on demand should be allowed (cluster default is false). On-demand CF loading doesn't load column families until necessary.
+   * @param load_column_families_on_demand
+   */
+  void SetLoadColumnFamiliesOnDemand(bool load_column_families_on_demand);
+
+  /**
+   * @brief Get the raw loadColumnFamiliesOnDemand setting.
+   */
+  bool LoadColumnFamiliesOnDemand() const;
+
+  /**
+   * @brief Get up to the specified number of versions of each column if specified else get default i.e. one.
+   * @param max_versions - maximum versions for each column.
+   */
+  Scan &SetMaxVersions(uint32_t max_versions = 1);
+
+  /**
+   * @brief the max number of versions to fetch
+   */
+  int MaxVersions() const;
+
+  /**
+   * @brief Set the maximum result size. The default is -1; this means that no specific maximum result size will be set for this scan, and the global configured value will be used instead. (Defaults to unlimited).
+   * @param The maximum result size in bytes.
+   */
+  void SetMaxResultSize(long max_result_size);
+
+  /**
+   * @brief the maximum result size in bytes.
+   */
+  long MaxResultSize() const;
+
+  /**
+   * @brief Get versions of columns only within the specified timestamp range, [min_stamp, max_stamp). Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the default.
+   * @param min_stamp - minimum timestamp value, inclusive.
+   * @param max_stamp - maximum timestamp value, exclusive.
+   */
+  Scan &SetTimeRange(long min_stamp, long max_stamp);
+
+  /**
+   * @brief Get versions of columns with the specified timestamp. Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the defaut.
+   * @param timestamp - version timestamp
+   */
+  Scan &SetTimeStamp(long timestamp);
+
+  /**
+   * @brief Return Timerange
+   */
+  const TimeRange& Timerange() const;
+
+  /**
+   * @brief Returns true if family map (FamilyMap) is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the Scan family map (FamilyMap) for this Scan operation.
+   */
+  const FamilyMap &Family() const;
+
+ private:
+  std::string start_row_ = "";
+  std::string stop_row_ = "";
+  uint32_t max_versions_ = 1;
+  int caching_ = -1;
+  long max_result_size_ = -1;
+  bool cache_blocks_ = true;
+  bool load_column_families_on_demand_ = false;
+  bool reversed_ = false;
+  bool small_ = false;
+  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_;
+
+  /**
+   * @brief Checks for row length validity, throws if length check fails, returns null otherwise.
+   * @param row - row whose validity needs to be checked
+   * @throws std::runtime_error if row length equals 0 or greater than std::numeric_limits<short>::max();
+   */
+  void CheckRow(const std::string &row);
+};
+} /* namespace hbase */