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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/configuration.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/configuration.h b/hbase-native-client/include/hbase/client/configuration.h
new file mode 100644
index 0000000..d70941c
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/configuration.h
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+/* Some pieces of code have been added from HDFS-8707 */
+#pragma once
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <experimental/optional>
+
+namespace hbase {
+
+template <class T>
+using optional = std::experimental::optional<T>;
+
+class Configuration {
+ public:
+  /**
+   * @brief Create Configuration object using an empty map. This
+   * object WILL NOT be initialized from reading XML configuration (hbase-site.xml, etc).
+   * Use HBaseConfigurationLoader to initialize the configuration from
+   * hbase-default.xml and hbase-site.xml files.
+   */
+  Configuration();
+
+  ~Configuration();
+
+  /**
+   * @brief Returns value identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. SubstituteVars will be
+   * called for any variable expansion.
+   */
+  std::string Get(const std::string &key, const std::string &default_value) const;
+
+  /**
+   * @brief Returns int32_t identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to int32_t fails
+   */
+  int32_t GetInt(const std::string &key, int32_t default_value) const;
+
+  /**
+   * @brief Returns int64_t identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called and
+   * @throws std::runtime_error if conversion to int64_t fails
+   */
+  int64_t GetLong(const std::string &key, int64_t default_value) const;
+
+  /**
+   * @brief Returns double identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to double fails
+   */
+  double GetDouble(const std::string &key, double default_value) const;
+
+  /**
+   * @brief Returns bool identified by key in ConfigMap else default value if
+   * property is absent.
+   * @param key Property whose value is to be fetched. Internally Get(key) will
+   * be called.
+   * @throws std::runtime_error if conversion to bool fails
+   */
+  bool GetBool(const std::string &key, bool default_value) const;
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void Set(const std::string &key, const std::string &value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetInt(const std::string &key, int32_t value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetLong(const std::string &key, int64_t value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetDouble(const std::string &key, double value);
+
+  /**
+   * @brief This method sets the given key to the value.
+   * @param key property name
+   * @param value property value
+   */
+  void SetBool(const std::string &key, bool value);
+
+ private:
+  friend class HBaseConfigurationLoader;
+
+  /* Transparent data holder for property values */
+  /* Same as Jira HDFS-8707 */
+  struct ConfigData {
+    std::string value;
+    bool final;
+    ConfigData() : final(false) {}
+    explicit ConfigData(const std::string &value) : value(value), final(false) {}
+    void operator=(const std::string &new_value) {
+      value = new_value;
+      final = false;
+    }
+  };
+
+  /**
+   * @brief Map which hold configuration properties.
+   */
+  using ConfigMap = std::map<std::string, ConfigData>;
+
+  /**
+   * @brief Create Configuration object using config_map;
+   * @param config_map - Map consisting of properties.
+   */
+  explicit Configuration(ConfigMap &config_map);
+
+  // Property map filled all the properties loaded by ConfigurationLoader
+  ConfigMap hb_property_;
+
+  // Variable expansion depth
+  const int kMaxSubsts = 20;
+
+  /**
+   * @brief returns value for a property identified by key in ConfigMap.
+   * SubstituteVars will be called for any variable expansion.
+   * @param key Key whose value is to be fetched.
+   */
+  optional<std::string> Get(const std::string &key) const;
+
+  /**
+   * @brief returns optional int32_t value identified by the key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to
+   * int32_t using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to int32_t fails
+   */
+  optional<int32_t> GetInt(const std::string &key) const;
+
+  /**
+   * @brief returns optional int64_t value identified by the key in ConfigMap.
+   * Get(key) is called internally to get the string value which is then
+   * converted to int64_t using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to int64_t fails
+   */
+  optional<int64_t> GetLong(const std::string &key) const;
+
+  /**
+   * @brief returns optional double value identified by the key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to
+   * double using boost::lexical_cast.
+   * @param key Key whose value is to be fetched.
+   * @throws std::runtime_error if conversion to double fails
+   */
+  optional<double> GetDouble(const std::string &key) const;
+
+  /**
+   * @brief returns optional bool for a property identified by key in ConfigMap.
+   * Get(key) is called to get the string value which is then converted to bool
+   * by checking the validity.
+   * @param key Key whose value is to be fetched. Get(key) is called to get the
+   * string value which is then converted to bool.
+   * @throws std::runtime_error if conversion to bool fails
+   */
+  optional<bool> GetBool(const std::string &key) const;
+
+  /**
+   * @brief This method will perform any variable expansion if present.
+   * @param expression expression string to perform substitutions on.
+   * @throws std::runtime_error if MAX_SUBSTS is exhausted and any more
+   * substitutions are reqd to be performed.
+   */
+  std::string SubstituteVars(const std::string &expr) const;
+
+  /**
+   * @brief This method will check if variable expansion has to be performed or
+   * not. Expression will be checked for presence of "${" and "}" to perform
+   * variable expansion.
+   * @param expr Expression on which will be checked for validity.
+   * @param sub_variable Extracted variable from expr which will be checked
+   * against environment value or ConfigMap values.
+   */
+  size_t IsSubVariable(const std::string &expr, std::string &sub_variable) const;
+
+  /**
+   * @brief This method will fetch value for key from environment if present.
+   * @param key key to be fetched from environment.
+   */
+  optional<std::string> GetEnv(const std::string &key) const;
+
+  /**
+   * @brief This method will fetch value for key from ConfigMap if present.
+   * @param key key to be fetched from environment.
+   */
+  optional<std::string> GetProperty(const std::string &key) const;
+};
+} /* namespace hbase */

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/delete.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/delete.h b/hbase-native-client/include/hbase/client/delete.h
new file mode 100644
index 0000000..5fdbcc1
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/delete.h
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Delete : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  /*
+   * If no further operations are done, this will delete everything
+   * associated with the specified row (all versions of all columns in all
+   * families), with timestamp from current point in time to the past.
+   * Cells defining timestamp for a future point in time
+   * (timestamp > current time) will not be deleted.
+   */
+  explicit Delete(const std::string& row) : Mutation(row) {}
+
+  Delete(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
+  Delete(const Delete& cdelete) : Mutation(cdelete) {}
+  Delete& operator=(const Delete& cdelete) {
+    Mutation::operator=(cdelete);
+    return *this;
+  }
+
+  ~Delete() = default;
+
+  /**
+   *  @brief Add the specified column to this Delete operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   */
+  Delete& AddColumn(const std::string& family, const std::string& qualifier);
+
+  /**
+   *  @brief Add the specified column to this Delete operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   */
+  Delete& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp);
+
+  /**
+   *  @brief Deletes all versions of the specified column
+   *  @param family family name
+   *  @param qualifier column qualifier
+   */
+  Delete& AddColumns(const std::string& family, const std::string& qualifier);
+  /**
+   *  @brief Deletes all versions of the specified column with a timestamp less than
+   * or equal to the specified timestamp
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   */
+  Delete& AddColumns(const std::string& family, const std::string& qualifier, int64_t timestamp);
+  /**
+   *  @brief Add the specified family to this Delete operation.
+   *  @param family family name
+   */
+  Delete& AddFamily(const std::string& family);
+
+  /**
+   *  @brief Deletes all columns of the specified family with a timestamp less than
+   * or equal to the specified timestamp
+   *  @param family family name
+   *  @param timestamp version timestamp
+   */
+  Delete& AddFamily(const std::string& family, int64_t timestamp);
+  /**
+   *  @brief Deletes all columns of the specified family with a timestamp
+   *   equal to the specified timestamp
+   *  @param family family name
+   *  @param timestamp version timestamp
+   */
+  Delete& AddFamilyVersion(const std::string& family, int64_t timestamp);
+  /**
+   * Advanced use only.
+   * Add an existing delete marker to this Delete object.
+   */
+  Delete& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/get.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/get.h b/hbase-native-client/include/hbase/client/get.h
new file mode 100644
index 0000000..b6a11f9
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/get.h
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/query.h"
+#include "hbase/client/row.h"
+#include "hbase/client/time-range.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class Get : public Row, public Query {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Get(const std::string& row);
+  Get(const Get& cget);
+  Get& operator=(const Get& cget);
+
+  ~Get();
+
+  /**
+   * @brief Returns the maximum number of values to fetch per CF
+   */
+  int MaxVersions() const;
+
+  /**
+   * @brief Get up to the specified number of versions of each column. default
+   * is 1.
+   * @param max_versions max_versons to set
+   */
+  Get& SetMaxVersions(int32_t max_versions = 1);
+
+  /**
+   * @brief Returns whether blocks should be cached for this Get operation.
+   */
+  bool CacheBlocks() const;
+
+  /**
+   * @brief Set whether blocks should be cached for this Get operation.
+   * @param cache_blocks to set
+   */
+  Get& SetCacheBlocks(bool cache_blocks);
+
+  /**
+   * @brief Returns the Get family map for this Get operation. Used
+   * for constructing Scan object with an already constructed Get
+   */
+  const std::map<std::string, std::vector<std::string>>& FamilyMap() const;
+
+  /**
+   * @brief Returns the timerange for this Get
+   */
+  const TimeRange& Timerange() const;
+
+  /**
+   * @brief Get versions of columns only within the specified timestamp range,
+   * [minStamp, maxStamp).
+   * @param minStamp the minimum timestamp, inclusive
+   * @param maxStamp the maximum timestamp, exclusive
+   */
+  Get& SetTimeRange(int64_t min_timestamp, int64_t max_timestamp);
+
+  /**
+   * @brief Get versions of columns with the specified timestamp.
+   * @param The timestamp to be set
+   */
+  Get& SetTimeStamp(int64_t timestamp);
+
+  /**
+   * @brief Get all columns from the specified family.
+   * @param family to be retrieved
+   */
+  Get& AddFamily(const std::string& family);
+
+  /**
+   *  @brief Get the column from the specific family with the specified
+   * qualifier.
+   *  @param family to be retrieved
+   *  @param qualifier to be retrieved
+   */
+  Get& AddColumn(const std::string& family, const std::string& qualifier);
+
+  /**
+   * @brief Returns true if family map is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the consistency level for this Get operation
+   */
+  hbase::pb::Consistency Consistency() const;
+
+  /**
+   * @brief Sets the consistency level for this Get operation
+   * @param Consistency to be set
+   */
+  Get& SetConsistency(hbase::pb::Consistency consistency);
+
+ private:
+  int32_t max_versions_ = 1;
+  bool cache_blocks_ = true;
+  bool check_existence_only_ = false;
+  std::map<std::string, std::vector<std::string>> family_map_;
+  hbase::pb::Consistency consistency_ = hbase::pb::Consistency::STRONG;
+  std::unique_ptr<TimeRange> tr_ = std::make_unique<TimeRange>();
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/hbase-configuration-loader.h b/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
new file mode 100644
index 0000000..8792aa2
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/hbase-configuration-loader.h
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <boost/optional.hpp>
+
+#include "hbase/client/configuration.h"
+#include "hbase/utils/optional.h"
+
+namespace hbase {
+
+class HBaseConfigurationLoader {
+ public:
+  HBaseConfigurationLoader();
+  ~HBaseConfigurationLoader();
+
+  /**
+   * @brief Creates a Configuration object based on default resources loaded
+   * from default search paths. Default search path will be either $HBASE_CONF
+   * is set or /etc/hbase/conf. Default resources are hbase-default.xml and
+   * hbase-site.xml.SetDefaultSearchPath() and AddDefaultResources() are used
+   * for the same.
+   * Values are loaded in from hbase-default.xml first and then from
+   * hbase-site.xml.
+   * Properties in hbase-site.xml will override the ones in hbase-default.xml
+   * unless marked as final
+   */
+  optional<Configuration> LoadDefaultResources();
+
+  /*
+   * @brief Creates a Configuration object based on resources loaded from search
+   * paths. Search paths are defined in search_path. Values are loaded from
+   * resources and will be overridden unless marked as final
+   * @param search_path - ':' search paths to load resources.
+   * @param resources - list of resources used to load configuration properties.
+   */
+  optional<Configuration> LoadResources(const std::string &search_path,
+                                        const std::vector<std::string> &resources);
+
+ private:
+  using ConfigMap = Configuration::ConfigMap;
+  const std::string kHBaseDefaultXml = "hbase-default.xml";
+  const std::string kHBaseSiteXml = "hbase-site.xml";
+  const std::string kHBaseDefauktConfPath = "/etc/hbase/conf";
+
+  // Adds FILE_SEPARATOR to the search path
+  const char kFileSeparator = '/';
+
+  // Separator using which multiple search paths can be defined.
+  const char kSearchPathSeparator = ':';
+
+  /**
+   * List of paths which will be looked up for loading properties.
+   */
+  std::vector<std::string> search_paths_;
+
+  /**
+   * List of files which will be looked up in search_paths_ to load properties.
+   */
+  std::vector<std::string> resources_;
+
+  /**
+   * @brief This method sets the search path to the default search path (i.e.
+   * "$HBASE_CONF" or "/etc/hbase/conf" if HBASE_CONF is absent)
+   */
+  void SetDefaultSearchPath();
+
+  /**
+   * @brief Clears out the set search path(s)
+   */
+  void ClearSearchPath();
+
+  /**
+   * @brief Sets the search path to ":"-delimited paths, clearing already
+   * defined values
+   * @param search_path Single path or ":"-delimited separated paths
+   */
+  void SetSearchPath(const std::string &search_path);
+
+  /**
+   * @brief Adds an element to the search path if not already present.
+   * @param search_path Path that will be added to load config values
+   */
+  void AddToSearchPath(const std::string &search_path);
+
+  /**
+   * @brief This method will add default resources i.e. hbase-default.xml and
+   * hbase-site.xml to the default search path.
+   */
+  void AddDefaultResources();
+
+  /**
+   * @brief Adds resources to list for loading config values.
+   * @param filename to be added to resources.
+   */
+  void AddResources(const std::string &filename);
+
+  /**
+   * @brief Loads properties in file identified by file in a map identified by
+   * property_map
+   * @param file XML file which defines HBase configuration properties.
+   * @param property_map Property map representing HBase configuration as key
+   * value pairs.
+   * @throws Boost ptree exception if some parsing issue occurs.
+   */
+  bool LoadProperties(const std::string &file, ConfigMap &property_map);
+
+  /**
+   * @brief This method will create a map of the name and properties.
+   * @param map Property map to hold configuration properties.
+   * @param key value of name node.
+   * @param value value of value node.
+   * @param final_text value of final node true or false if present
+   */
+  bool UpdateMapWithValue(ConfigMap &map, const std::string &key, const std::string &value,
+                          boost::optional<std::string> final_text);
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/hbase-rpc-controller.h b/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
new file mode 100644
index 0000000..33f552b
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/hbase-rpc-controller.h
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <google/protobuf/service.h>
+#include <chrono>
+#include <string>
+
+namespace hbase {
+
+class HBaseRpcController : public google::protobuf::RpcController {
+ public:
+  HBaseRpcController() {}
+  virtual ~HBaseRpcController() = default;
+
+  void set_call_timeout(const std::chrono::milliseconds& call_timeout) {
+    // TODO:
+  }
+
+  void Reset() override {}
+
+  bool Failed() const override { return false; }
+
+  folly::exception_wrapper exception() { return exception_; }
+
+  void set_exception(const folly::exception_wrapper& exception) { exception_ = exception; }
+
+  std::string ErrorText() const override { return ""; }
+
+  void StartCancel() override {}
+
+  void SetFailed(const std::string& reason) override {}
+
+  bool IsCanceled() const override { return false; }
+
+  void NotifyOnCancel(google::protobuf::Closure* callback) override {}
+
+ private:
+  folly::exception_wrapper exception_;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/increment.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/increment.h b/hbase-native-client/include/hbase/client/increment.h
new file mode 100644
index 0000000..3169c19
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/increment.h
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Increment : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Increment(const std::string& row) : Mutation(row) {}
+  Increment(const Increment& cincrement) : Mutation(cincrement) {}
+  Increment& operator=(const Increment& cincrement) {
+    Mutation::operator=(cincrement);
+    return *this;
+  }
+
+  ~Increment() = default;
+
+  /**
+   *  @brief Increment the column from the specific family with the specified qualifier
+   * by the specified amount.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param amount amount to increment by
+   */
+  Increment& AddColumn(const std::string& family, const std::string& qualifier, int64_t amount);
+  Increment& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/keyvalue-codec.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/keyvalue-codec.h b/hbase-native-client/include/hbase/client/keyvalue-codec.h
new file mode 100644
index 0000000..5a92c8c
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/keyvalue-codec.h
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <folly/io/Cursor.h>
+#include <folly/io/IOBuf.h>
+#include <memory>
+
+#include "hbase/client/cell.h"
+#include "hbase/serde/codec.h"
+
+namespace hbase {
+
+/**
+ * @brief Class for parsing sequence of Cells based on org.apache.hadoop.hbase.KeyValueCodec.java
+ *
+ * KeyValueCodec implements CellScanner interface. Sequence of cells are obtained from cell_block.
+ * We have CreateEncoder and CreateDecoder public methods which will return Encoder/Decoder
+ * instances which will be used to obtain individual cells in cell_block.
+ * Usage:-
+ * 1) Cell Decoding:-
+ * unique_ptr<CellScanner> cell_scanner = KeyValueCodec::CreateDecoder(cell_block, cb_start_offset,
+ *    cb_length);
+ * while (cell_scanner->Advance()) {
+ *  auto current_cell = cell_scanner->Current
+ * }
+ */
+class KeyValueCodec : public Codec {
+ public:
+  /**
+        * Constructor
+        */
+  KeyValueCodec() {}
+
+  std::unique_ptr<Codec::Encoder> CreateEncoder() override { return std::make_unique<KVEncoder>(); }
+  std::unique_ptr<Codec::Decoder> CreateDecoder(std::unique_ptr<folly::IOBuf> cell_block,
+                                                uint32_t offset, uint32_t length) override {
+    return std::make_unique<KVDecoder>(std::move(cell_block), offset, length);
+  }
+
+  /** @brief returns the java class name corresponding to this Codec implementation */
+  virtual const char* java_class_name() const override { return kJavaClassName; }
+
+  static constexpr const char* kJavaClassName = "org.apache.hadoop.hbase.codec.KeyValueCodec";
+
+ private:
+  class KVEncoder : public Codec::Encoder {
+   public:
+    KVEncoder() {}
+
+    void Write(const Cell& cell) {
+      // TODO: Encode Cells using KeyValueCodec wire format
+    }
+
+    void Flush() {}
+  };
+
+  class KVDecoder : public Codec::Decoder {
+   public:
+    KVDecoder(std::unique_ptr<folly::IOBuf> cell_block, uint32_t cell_block_start_offset,
+              uint32_t cell_block_length);
+    ~KVDecoder();
+
+    /**
+     * @brief Overridden from CellScanner. This method parses cell_block and stores the current in
+     * current_cell_. Current cell can be obtained using cell_scanner.Current();
+     */
+    bool Advance();
+
+    /**
+     * @brief returns the current cell
+     */
+    const std::shared_ptr<Cell> Current() const { return current_cell_; }
+
+    /**
+     * @brief returns the total length of cell_meta_block
+     */
+    uint32_t CellBlockLength() const;
+
+   private:
+    std::shared_ptr<Cell> Decode(folly::io::Cursor& cursor);
+
+    /**
+     * Size of boolean in bytes
+     */
+    const int kHBaseSizeOfBoolean_ = sizeof(uint8_t) / sizeof(uint8_t);
+
+    /**
+     * Size of byte in bytes
+     */
+    const uint8_t kHBaseSizeOfByte_ = kHBaseSizeOfBoolean_;
+
+    /**
+     * Size of int in bytes
+     */
+    const uint32_t kHBaseSizeOfInt_ = sizeof(uint32_t) / kHBaseSizeOfByte_;
+
+    /**
+     * Size of long in bytes
+     */
+    const uint64_t kHBaseSizeOfLong_ = sizeof(uint64_t) / kHBaseSizeOfByte_;
+
+    /**
+     * Size of Short in bytes
+     */
+    const uint16_t kHBaseSizeOfShort_ = sizeof(uint16_t) / kHBaseSizeOfByte_;
+
+    const uint32_t kHBaseSizeOfKeyLength_ = kHBaseSizeOfInt_;
+    const uint32_t kHBaseSizeOfValueLength_ = kHBaseSizeOfInt_;
+    const uint16_t kHBaseSizeOfRowLength_ = kHBaseSizeOfShort_;
+    const uint8_t kHBaseSizeOfFamilyLength_ = kHBaseSizeOfByte_;
+    const uint64_t kHBaseSizeOfTimestamp_ = kHBaseSizeOfLong_;
+    const uint8_t kHBaseSizeOfKeyType_ = kHBaseSizeOfByte_;
+    const uint32_t kHBaseSizeOfTimestampAndKey_ = kHBaseSizeOfTimestamp_ + kHBaseSizeOfKeyType_;
+    const uint32_t kHBaseSizeOfKeyInfrastructure_ =
+        kHBaseSizeOfRowLength_ + kHBaseSizeOfFamilyLength_ + kHBaseSizeOfTimestampAndKey_;
+    const uint32_t kHBaseSizeOfKeyValueInfrastructure_ =
+        kHBaseSizeOfKeyLength_ + kHBaseSizeOfValueLength_;
+
+    std::unique_ptr<folly::IOBuf> cell_block_ = nullptr;
+    uint32_t offset_ = 0;
+    uint32_t length_ = 0;
+    uint32_t cur_pos_ = 0;
+    bool end_of_cell_block_ = false;
+
+    std::shared_ptr<Cell> current_cell_ = nullptr;
+  };
+};
+
+} /* namespace hbase */

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/meta-utils.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/meta-utils.h b/hbase-native-client/include/hbase/client/meta-utils.h
new file mode 100644
index 0000000..3fc320a
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/meta-utils.h
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "hbase/connection/request.h"
+#include "hbase/connection/response.h"
+#include "hbase/client/region-location.h"
+#include "hbase/if/HBase.pb.h"
+#include "hbase/serde/table-name.h"
+
+namespace hbase {
+
+/**
+ * @brief Utility for meta operations.
+ */
+class MetaUtil {
+ public:
+  static constexpr const char *kSystemNamespace = "hbase";
+  static constexpr const char *kMetaTableQualifier = "meta";
+  static constexpr const char *kMetaTableName = "hbase:meta";
+  static constexpr const char *kMetaRegion = "1588230740";
+  static constexpr const char *kMetaRegionName = "hbase:meta,,1";
+  static constexpr const char *kCatalogFamily = "info";
+  static constexpr const char *kRegionInfoColumn = "regioninfo";
+  static constexpr const char *kServerColumn = "server";
+
+  MetaUtil();
+
+  /**
+   * Given a table and a row give the row key from which to start a scan to find
+   * region locations.
+   */
+  std::string RegionLookupRowkey(const hbase::pb::TableName &tn, const std::string &row) const;
+
+  /**
+   * Given a row we're trying to access create a request to look up the
+   * location.
+   */
+  std::unique_ptr<Request> MetaRequest(const hbase::pb::TableName tn, const std::string &row) const;
+
+  /**
+   * Return a RegionLocation from the parsed Response
+   */
+  std::shared_ptr<RegionLocation> CreateLocation(const Response &resp,
+                                                 const hbase::pb::TableName &tn);
+
+  /**
+   * Return whether the table is the meta table.
+   */
+  static bool IsMeta(const hbase::pb::TableName &tn);
+
+  const pb::RegionInfo &meta_region_info() const { return meta_region_info_; }
+
+ private:
+  pb::RegionInfo meta_region_info_;
+};
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/multi-response.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/multi-response.h b/hbase-native-client/include/hbase/client/multi-response.h
new file mode 100644
index 0000000..b9b9a38
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/multi-response.h
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <folly/ExceptionWrapper.h>
+#include <exception>
+#include <map>
+#include <memory>
+#include <string>
+
+#include "hbase/client/region-result.h"
+#include "hbase/client/result.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class MultiResponse {
+ public:
+  MultiResponse();
+  /**
+   * @brief Returns Number of pairs in this container
+   */
+  int Size() const;
+
+  /**
+   * Add the pair to the container, grouped by the regionName
+   *
+   * @param regionName
+   * @param originalIndex the original index of the Action (request).
+   * @param resOrEx the result or error; will be empty for successful Put and Delete actions.
+   */
+  void AddRegionResult(const std::string& region_name, int32_t original_index,
+                       std::shared_ptr<Result> result,
+                       std::shared_ptr<folly::exception_wrapper> exc);
+
+  void AddRegionException(const std::string& region_name,
+                          std::shared_ptr<folly::exception_wrapper> exception);
+
+  /**
+   * @return the exception for the region, if any. Null otherwise.
+   */
+  std::shared_ptr<folly::exception_wrapper> RegionException(const std::string& region_name) const;
+
+  const std::map<std::string, std::shared_ptr<folly::exception_wrapper>>& RegionExceptions() const;
+
+  void AddStatistic(const std::string& region_name, std::shared_ptr<pb::RegionLoadStats> stat);
+
+  const std::map<std::string, std::shared_ptr<RegionResult>>& RegionResults() const;
+
+  ~MultiResponse();
+
+ private:
+  // map of regionName to map of Results by the original index for that Result
+  std::map<std::string, std::shared_ptr<RegionResult>> results_;
+  /**
+   * The server can send us a failure for the region itself, instead of individual failure.
+   * It's a part of the protobuf definition.
+   */
+  std::map<std::string, std::shared_ptr<folly::exception_wrapper>> exceptions_;
+};
+
+} /* namespace hbase */

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/mutation.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/mutation.h b/hbase-native-client/include/hbase/client/mutation.h
new file mode 100644
index 0000000..f70e2b7
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/mutation.h
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <limits>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/row.h"
+#include "hbase/if/Client.pb.h"
+
+namespace hbase {
+
+class Mutation : public Row {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Mutation(const std::string& row);
+  Mutation(const std::string& row, int64_t timestamp);
+  Mutation(const Mutation& cmutation);
+  Mutation& operator=(const Mutation& cmutation);
+
+  virtual ~Mutation() = default;
+
+  /**
+   * @brief Returns the Mutation family map for this operation.
+   */
+  const std::map<std::string, std::vector<std::unique_ptr<Cell>>>& FamilyMap() const {
+    return family_map_;
+  }
+
+  /**
+   * @brief Returns the timerange for this Get
+   */
+  int64_t TimeStamp() const { return timestamp_; }
+
+  /**
+   * @brief Get versions of columns with the specified timestamp.
+   * @param The timestamp to be set
+   */
+  Mutation& SetTimeStamp(int64_t timestamp) {
+    timestamp_ = timestamp;
+    return *this;
+  }
+
+  /**
+   * @brief Returns true if family map is non empty false otherwise
+   */
+  bool HasFamilies() const;
+
+  /**
+   * @brief Returns the durability level for this Mutation operation
+   */
+  pb::MutationProto_Durability Durability() const;
+
+  /**
+   * @brief Sets the durability level for this Mutation operation
+   * @param durability  the durability to be set
+   */
+  Mutation& SetDurability(pb::MutationProto_Durability durability);
+
+ public:
+  static const constexpr int64_t kLatestTimestamp = std::numeric_limits<int64_t>::max();
+
+ protected:
+  std::map<std::string, std::vector<std::unique_ptr<Cell>>> family_map_;
+  pb::MutationProto_Durability durability_ =
+      hbase::pb::MutationProto_Durability::MutationProto_Durability_USE_DEFAULT;
+  int64_t timestamp_ = kLatestTimestamp;
+
+  std::unique_ptr<Cell> CreateCell(const std::string& family, const std::string& qualifier,
+                                   int64_t timestamp, const std::string& value);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/put.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/put.h b/hbase-native-client/include/hbase/client/put.h
new file mode 100644
index 0000000..cb0e17d
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/put.h
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+#include "hbase/client/cell.h"
+#include "hbase/client/mutation.h"
+
+namespace hbase {
+
+class Put : public Mutation {
+ public:
+  /**
+   * Constructors
+   */
+  explicit Put(const std::string& row) : Mutation(row) {}
+  Put(const std::string& row, int64_t timestamp) : Mutation(row, timestamp) {}
+  Put(const Put& cput) : Mutation(cput) {}
+  Put& operator=(const Put& cput) {
+    Mutation::operator=(cput);
+    return *this;
+  }
+
+  ~Put() = default;
+
+  /**
+   *  @brief Add the specified column and value to this Put operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param value column value
+   */
+  Put& AddColumn(const std::string& family, const std::string& qualifier, const std::string& value);
+
+  /**
+   *  @brief Add the specified column and value to this Put operation.
+   *  @param family family name
+   *  @param qualifier column qualifier
+   *  @param timestamp version timestamp
+   *  @param value column value
+   */
+  Put& AddColumn(const std::string& family, const std::string& qualifier, int64_t timestamp,
+                 const std::string& value);
+
+  Put& Add(std::unique_ptr<Cell> cell);
+};
+
+}  // namespace hbase

http://git-wip-us.apache.org/repos/asf/hbase/blob/128fc306/hbase-native-client/include/hbase/client/query.h
----------------------------------------------------------------------
diff --git a/hbase-native-client/include/hbase/client/query.h b/hbase-native-client/include/hbase/client/query.h
new file mode 100644
index 0000000..76f613e
--- /dev/null
+++ b/hbase-native-client/include/hbase/client/query.h
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+#pragma once
+
+#include <memory>
+
+#include "hbase/client/filter.h"
+
+namespace hbase {
+
+/**
+ * Base class for read RPC calls (Get / Scan).
+ */
+class Query {
+ public:
+  Query() = default;
+  Query(const Query &query) {
+    // filter can be a custom subclass of Filter, so we do not do a deep copy here.
+    filter_ = query.filter_;
+  }
+
+  Query &operator=(const Query &query) {
+    filter_ = query.filter_;
+    return *this;
+  }
+
+  virtual ~Query() {}
+
+  void SetFilter(std::shared_ptr<Filter> filter) { filter_ = filter; }
+
+  const std::shared_ptr<Filter> filter() const { return filter_; }
+
+ protected:
+  std::shared_ptr<Filter> filter_ = nullptr;
+};
+
+}  // namespace hbase