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

[hbase] 42/133: HBASE-16489 Configuration parsing (Sudeep Sunthankar)

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

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

commit 13275532f8b63760cc59bac4e731ce2cd2a15513
Author: Enis Soztutar <en...@apache.org>
AuthorDate: Wed Nov 30 14:57:18 2016 -0800

    HBASE-16489 Configuration parsing (Sudeep Sunthankar)
---
 hbase-native-client/core/BUCK                      |  10 +
 hbase-native-client/core/configuration.cc          | 231 ++++++++++++
 hbase-native-client/core/configuration.h           | 191 ++++++++++
 .../core/hbase_configuration-test.cc               | 388 +++++++++++++++++++++
 .../core/hbase_configuration_loader.cc             | 214 ++++++++++++
 .../core/hbase_configuration_loader.h              | 146 ++++++++
 6 files changed, 1180 insertions(+)

diff --git a/hbase-native-client/core/BUCK b/hbase-native-client/core/BUCK
index 7e9044a..9dbc82e 100644
--- a/hbase-native-client/core/BUCK
+++ b/hbase-native-client/core/BUCK
@@ -29,6 +29,8 @@ cxx_library(
         "meta-utils.h",
         "get.h",
         "time_range.h",
+        "configuration.h",
+        "hbase_configuration_loader.h",
     ],
     srcs=[
         "cell.cc",
@@ -37,6 +39,8 @@ cxx_library(
         "meta-utils.cc",
         "get.cc",
         "time_range.cc",
+        "configuration.cc",
+        "hbase_configuration_loader.cc",
     ],
     deps=[
         "//connection:connection",
@@ -75,6 +79,12 @@ cxx_test(name="time_range-test",
          ],
          deps=[":core", ],
          run_test_separately=True, )
+cxx_test(name="hbase_configuration-test",
+         srcs=[
+             "hbase_configuration-test.cc",
+         ],
+         deps=[":core", ],
+         run_test_separately=True, )
 cxx_binary(name="simple-client",
            srcs=["simple-client.cc", ],
            deps=[":core", "//connection:connection"], )
diff --git a/hbase-native-client/core/configuration.cc b/hbase-native-client/core/configuration.cc
new file mode 100644
index 0000000..9816cff
--- /dev/null
+++ b/hbase-native-client/core/configuration.cc
@@ -0,0 +1,231 @@
+/*
+ * 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 "configuration.h"
+
+#include <stdexcept>
+
+#include <boost/lexical_cast.hpp>
+#include <glog/logging.h>
+
+namespace hbase {
+
+Configuration::Configuration(ConfigMap &config_map)
+    : hb_property_(std::move(config_map)) {}
+
+Configuration::~Configuration() {}
+
+size_t Configuration::IsSubVariable(const std::string &expr,
+                                    std::string &sub_variable) const {
+  size_t start_pos = expr.find("${");
+  if (std::string::npos != start_pos) {
+    size_t pos_next = expr.find("}", start_pos + 1);
+    if (std::string::npos != pos_next) {
+      sub_variable = expr.substr(start_pos + 2, pos_next - (start_pos + 2));
+    }
+  }
+  return start_pos;
+}
+
+std::string Configuration::SubstituteVars(const std::string &expr) const {
+  if (0 == expr.size()) return expr;
+
+  std::string eval(expr);
+  std::string value_to_be_replaced("");
+  std::string var("");
+  for (int i = 0; i < kMaxSubsts; i++) {
+    var = "";
+    size_t start_pos = IsSubVariable(eval, var);
+    if (start_pos != std::string::npos) {
+      // We are blindly checking for environment property at first.
+      // If we don't get any value from GetEnv, check in hbase-site.xml.
+      value_to_be_replaced =
+          GetEnv(var).value_or(GetProperty(var).value_or(""));
+
+      // we haven't found any value yet so we are returning eval
+      if (0 == value_to_be_replaced.size()) {
+        return eval;
+      }
+
+      // return original expression if there is a loop
+      if (value_to_be_replaced == expr) {
+        return expr;
+      }
+
+      eval.replace(start_pos, var.size() + 3, value_to_be_replaced);
+
+    } else {
+      // No further expansion required.
+      return eval;
+    }
+  }
+  // We reached here if the loop is exhausted
+  // If MAX_SUBSTS is exhausted, check if more variable substitution is reqd.
+  // If any-more substitutions are reqd, throw an error.
+  var = "";
+  if (IsSubVariable(eval, var) != std::string::npos) {
+    throw std::runtime_error("Variable substitution depth too large: " +
+                             std::to_string(kMaxSubsts) + " " + expr);
+  } else {
+    return eval;
+  }
+}
+
+optional<std::string> Configuration::GetEnv(const std::string &key) const {
+  char buf[2048];
+
+  if ("user.name" == key) {
+#ifdef HAVE_GETLOGIN
+    return std::experimental::make_optional(getlogin());
+#else
+    DLOG(WARNING) << "Client user.name not implemented";
+    return optional<std::string>();
+#endif
+  }
+
+  if ("user.dir" == key) {
+#ifdef HAVE_GETCWD
+    if (getcwd(buf, sizeof(buf))) {
+      return std::experimental::make_optional(buf);
+    } else {
+      return optional<std::string>();
+    }
+#else
+    DLOG(WARNING) << "Client user.dir not implemented";
+    return optional<std::string>();
+#endif
+  }
+
+  if ("user.home" == key) {
+#if defined(HAVE_GETUID) && defined(HAVE_GETPWUID_R)
+    uid = getuid();
+    if (!getpwuid_r(uid, &pw, buf, sizeof(buf), &pwp)) {
+      return std::experimental::make_optional(buf);
+    } else {
+      return optional<std::string>();
+    }
+#else
+    DLOG(WARNING) << "Client user.home not implemented";
+    return optional<std::string>();
+#endif
+  }
+  return optional<std::string>();
+}
+
+optional<std::string> Configuration::GetProperty(const std::string &key) const {
+
+  auto found = hb_property_.find(key);
+  if (found != hb_property_.end()) {
+    return std::experimental::make_optional(found->second.value);
+  } else {
+    return optional<std::string>();
+  }
+}
+
+optional<std::string> Configuration::Get(const std::string &key) const {
+  optional<std::string> raw = GetProperty(key);
+  if (raw) {
+    return std::experimental::make_optional(SubstituteVars(*raw));
+  } else {
+    return optional<std::string>();
+  }
+}
+
+std::string Configuration::Get(const std::string &key,
+                               const std::string &default_value) const {
+  return Get(key).value_or(default_value);
+}
+
+optional<int32_t> Configuration::GetInt(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(
+          boost::lexical_cast<int32_t>(*raw));
+    }
+    catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<int32_t>();
+}
+
+int32_t Configuration::GetInt(const std::string &key,
+                              int32_t default_value) const {
+  return GetInt(key).value_or(default_value);
+}
+
+optional<int64_t> Configuration::GetLong(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(
+          boost::lexical_cast<int64_t>(*raw));
+    }
+    catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<int64_t>();
+}
+
+int64_t Configuration::GetLong(const std::string &key,
+                               int64_t default_value) const {
+  return GetLong(key).value_or(default_value);
+}
+
+optional<double> Configuration::GetDouble(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    try {
+      return std::experimental::make_optional(
+          boost::lexical_cast<double>(*raw));
+    }
+    catch (const boost::bad_lexical_cast &blex) {
+      throw std::runtime_error(blex.what());
+    }
+  }
+  return optional<double>();
+}
+
+double Configuration::GetDouble(const std::string &key,
+                                double default_value) const {
+  return GetDouble(key).value_or(default_value);
+}
+
+optional<bool> Configuration::GetBool(const std::string &key) const {
+  optional<std::string> raw = Get(key);
+  if (raw) {
+    if (!strcasecmp((*raw).c_str(), "true")) {
+      return std::experimental::make_optional(true);
+    } else if (!strcasecmp((*raw).c_str(), "false")) {
+      return std::experimental::make_optional(false);
+    } else {
+      throw std::runtime_error(
+          "Unexpected value found while conversion to bool.");
+    }
+  }
+  return optional<bool>();
+}
+
+bool Configuration::GetBool(const std::string &key, bool default_value) const {
+  return GetBool(key).value_or(default_value);
+}
+
+} /* namespace hbase */
diff --git a/hbase-native-client/core/configuration.h b/hbase-native-client/core/configuration.h
new file mode 100644
index 0000000..44d6cd6
--- /dev/null
+++ b/hbase-native-client/core/configuration.h
@@ -0,0 +1,191 @@
+/*
+ * 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:
+  ~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;
+
+ 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) {};
+    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.
+   */
+  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 */
diff --git a/hbase-native-client/core/hbase_configuration-test.cc b/hbase-native-client/core/hbase_configuration-test.cc
new file mode 100644
index 0000000..167f506
--- /dev/null
+++ b/hbase-native-client/core/hbase_configuration-test.cc
@@ -0,0 +1,388 @@
+/*
+ * 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 <fstream>
+#include <iostream>
+
+#include <boost/filesystem.hpp>
+#include <gtest/gtest.h>
+#include <glog/logging.h>
+#include "core/hbase_configuration_loader.h"
+#include "core/configuration.h"
+
+using namespace hbase;
+
+const std::string kDefHBaseConfPath("./build/test-data/hbase-configuration-test/conf/");
+const std::string kHBaseConfPath("./build/test-data/hbase-configuration-test/custom-conf/");
+
+const std::string kHBaseDefaultXml("hbase-default.xml");
+const std::string kHBaseSiteXml("hbase-site.xml");
+
+const std::string kHBaseDefaultXmlData(
+    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
+    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
+    "Software Foundation (ASF) under one\n * or more contributor license "
+    "agreements.  See the NOTICE file\n * distributed with this work for "
+    "additional information\n * regarding copyright ownership.  The ASF "
+    "licenses this file\n * to you under the Apache License, Version 2.0 "
+    "(the\n * \"License\"); you may not use this file except in compliance\n * "
+    "with the License.  You may obtain a copy of the License at\n *\n *     "
+    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
+    "applicable law or agreed to in writing, software\n * distributed under "
+    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
+    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
+    "for the specific language governing permissions and\n * limitations under "
+    "the License.\n "
+    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
+    "name>\n<value>/root/hbase-docker/apps/hbase/data</value>\n<final>true</"
+    "final>\n</"
+    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
+    "name>\n<value>This value will be "
+    "overwritten</value>\n<final>false</final>\n</"
+    "property>\n\n<property>\n<name>default-prop</name>\n<value>default-value</"
+    "value>\n</property>\n\n</configuration>");
+const std::string kHBaseSiteXmlData(
+    "<?xml version=\"1.0\"?>\n<?xml-stylesheet type=\"text/xsl\" "
+    "href=\"configuration.xsl\"?>\n<!--\n/**\n *\n * Licensed to the Apache "
+    "Software Foundation (ASF) under one\n * or more contributor license "
+    "agreements.  See the NOTICE file\n * distributed with this work for "
+    "additional information\n * regarding copyright ownership.  The ASF "
+    "licenses this file\n * to you under the Apache License, Version 2.0 "
+    "(the\n * \"License\"); you may not use this file except in compliance\n * "
+    "with the License.  You may obtain a copy of the License at\n *\n *     "
+    "http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by "
+    "applicable law or agreed to in writing, software\n * distributed under "
+    "the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES "
+    "OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License "
+    "for the specific language governing permissions and\n * limitations under "
+    "the License.\n "
+    "*/\n-->\n<configuration>\n\n<property>\n<name>hbase.rootdir</"
+    "name>\n<value>This value will not be be "
+    "overwritten</value>\n</"
+    "property>\n\n<property>\n<name>hbase.zookeeper.property.datadir</"
+    "name>\n<value>/root/hbase-docker/zookeeper</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.name</"
+    "name>\n<value>${user.name}</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.dir</"
+    "name>\n<value>${user.dir}</value>\n</"
+    "property>\n\n<property>\n<name>hbase-client.user.home</"
+    "name>\n<value>${user.home}</value>\n</"
+    "property>\n\n<property>\n<name>selfRef</name>\n<value>${selfRef}</"
+    "value>\n</property>\n\n<property>\n<name>foo.substs</"
+    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
+    "bar},${bar},</value>\n</"
+    "property>\n\n<property>\n<name>foo.substs.exception</"
+    "name>\n<value>${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${"
+    "bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar},${bar}"
+    ",${bar},${bar}</value>\n</property>\n\n<property>\n<name>bar</"
+    "name>\n<value>bar-value</value>\n</"
+    "property>\n\n<property>\n<name>custom-prop</name>\n<value>custom-value</"
+    "value>\n</property>\n\n<property>\n<name>int</name>\n<value>16000</"
+    "value>\n</property>\n\n<property>\n<name>int.largevalue</"
+    "name>\n<value>2147483646</value>\n</"
+    "property>\n\n<property>\n<name>int.exception</name>\n<value>2147483648</"
+    "value>\n</property>\n\n<property>\n<name>long</name>\n<value>2147483850</"
+    "value>\n</property>\n\n<property>\n<name>long.largevalue</"
+    "name>\n<value>9223372036854775807</value>\n</"
+    "property>\n\n<property>\n<name>long.exception</"
+    "name>\n<value>9223372036854775810</value>\n</"
+    "property>\n\n<property>\n<name>double</name>\n<value>17.9769e+100</"
+    "value>\n</property>\n\n<property>\n<name>double.largevalue</"
+    "name>\n<value>170.769e+200</value>\n</"
+    "property>\n\n<property>\n<name>double.exception</"
+    "name>\n<value>1.79769e+310</value>\n</"
+    "property>\n\n<property>\n<name>bool.true</name>\n<value>true</value>\n</"
+    "property>\n\n<property>\n<name>bool.false</name>\n<value>false</value>\n</"
+    "property>\n\n<property>\n<name>bool.exception</name>\n<value>unknown "
+    "bool</value>\n</property>\n\n</configuration>");
+
+void WriteDataToFile(const std::string &file, const std::string &xml_data) {
+  std::ofstream hbase_conf;
+  hbase_conf.open(file.c_str());
+  hbase_conf << xml_data;
+  hbase_conf.close();
+}
+
+void CreateHBaseConf(const std::string &dir, const std::string &file,
+                     const std::string xml_data) {
+  // Directory will be created if not present
+  if (!boost::filesystem::exists(dir)) {
+    boost::filesystem::create_directories(dir);
+  }
+  // Remove temp file always
+  boost::filesystem::remove((dir + file).c_str());
+  WriteDataToFile((dir + file), xml_data);
+}
+
+void CreateHBaseConfWithEnv() {
+
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 1);
+}
+
+/*
+ * Config will be loaded from $HBASE_CONF. We set it @ kDefHBaseConfPath
+ * Config values will be loaded from hbase-default.xml and hbase-site.xml
+ * present in the above path.
+ */
+TEST(Configuration, LoadConfFromDefaultLocation) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  setenv("HBASE_CONF", kDefHBaseConfPath.c_str(), 0);
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value").c_str(),
+               "custom-value");
+  EXPECT_STREQ((*conf).Get("default-prop", "Set this value").c_str(),
+               "default-value");
+}
+
+/*
+ * Config will be loaded from hbase-site.xml defined at
+ * kHBaseConfPath
+ */
+TEST(Configuration, LoadConfFromCustomLocation) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+
+  HBaseConfigurationLoader loader;
+  std::vector<std::string> resources{kHBaseSiteXml};
+  hbase::optional<Configuration> conf =
+      loader.LoadResources(kHBaseConfPath, resources);
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
+  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
+}
+
+/*
+ * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
+ * kDefHBaseConfPath and kHBaseConfPath respectively. 
+ */
+TEST(Configuration, LoadConfFromMultipleLocatons) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kDefHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+  CreateHBaseConf(kHBaseConfPath, kHBaseDefaultXml, kHBaseDefaultXmlData);
+  CreateHBaseConf(kHBaseConfPath, kHBaseSiteXml, kHBaseSiteXmlData);
+
+  HBaseConfigurationLoader loader;
+  std::string conf_paths = kDefHBaseConfPath + ":" + kHBaseConfPath;
+  std::vector<std::string> resources{kHBaseDefaultXml, kHBaseSiteXml};
+  hbase::optional<Configuration> conf =
+      loader.LoadResources(conf_paths, resources);
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("default-prop", "From hbase-default.xml").c_str(),
+               "default-value");
+  EXPECT_STREQ((*conf).Get("custom-prop", "").c_str(), "custom-value");
+  EXPECT_STRNE((*conf).Get("custom-prop", "").c_str(), "some-value");
+}
+
+/*
+ * Config will be loaded from hbase-defualt.xml and hbase-site.xml @
+ * $HBASE_CONF.
+ * We set HBASE_CONF to kDefHBaseConfPath
+ * Below tests load the conf files in the same way unless specified.
+ */
+TEST(Configuration, DefaultValues) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("default-prop", "Set this value.").c_str(),
+               "default-value");
+  EXPECT_STREQ((*conf).Get("custom-prop", "Set this value.").c_str(),
+               "custom-value");
+}
+
+TEST(Configuration, FinalValues) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("hbase.rootdir", "").c_str(),
+               "/root/hbase-docker/apps/hbase/data");
+  EXPECT_STREQ((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
+               "/root/hbase-docker/zookeeper");
+  EXPECT_STRNE((*conf).Get("hbase.rootdir", "").c_str(),
+               "This value will not be be overwritten");
+  EXPECT_STRNE((*conf).Get("hbase.zookeeper.property.datadir", "").c_str(),
+               "This value will be overwritten");
+}
+
+/*
+ * Config will be loaded from HBASE_CONF which we set in
+ * CreateHBaseConfWithEnv().
+ * Config values will be loaded from hbase-default.xml and hbase-site.xml in the
+ * above path.
+ */
+TEST(Configuration, EnvVars) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("hbase-client.user.name", "").c_str(),
+               "${user.name}");
+  EXPECT_STRNE((*conf).Get("hbase-client.user.name", "root").c_str(),
+               "test-user");
+}
+
+TEST(Configuration, SelfRef) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("selfRef", "${selfRef}").c_str(), "${selfRef}");
+}
+
+TEST(Configuration, VarExpansion) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_STREQ((*conf).Get("foo.substs", "foo-value").c_str(),
+               "bar-value,bar-value,bar-value,bar-value,bar-value,bar-value,"
+               "bar-value,bar-value,bar-value,bar-value,");
+  EXPECT_STRNE((*conf).Get("foo.substs", "foo-value").c_str(), "bar-value");
+}
+
+TEST(Configuration, VarExpansionException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  ASSERT_THROW((*conf).Get("foo.substs.exception", "foo-value").c_str(),
+               std::runtime_error);
+}
+
+TEST(Configuration, GetInt) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_EQ(16000, (*conf).GetInt("int", 0));
+  EXPECT_EQ(2147483646, (*conf).GetInt("int.largevalue", 0));
+}
+
+TEST(Configuration, GetLong) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_EQ(2147483850, (*conf).GetLong("long", 0));
+  EXPECT_EQ(9223372036854775807, (*conf).GetLong("long.largevalue", 0));
+}
+
+TEST(Configuration, GetDouble) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_DOUBLE_EQ(17.9769e+100, (*conf).GetDouble("double", 0.0));
+  EXPECT_DOUBLE_EQ(170.769e+200, (*conf).GetDouble("double.largevalue", 0.0));
+}
+
+TEST(Configuration, GetBool) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  EXPECT_EQ(true, (*conf).GetBool("bool.true", true));
+  EXPECT_EQ(false, (*conf).GetBool("bool.false", false));
+}
+
+TEST(Configuration, GetIntException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetInt("int.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetLongException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetLong("long.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetDoubleException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetDouble("double.exception", 0), std::runtime_error);
+}
+
+TEST(Configuration, GetBoolException) {
+  // Remove already configured env if present.
+  unsetenv("HBASE_CONF");
+  CreateHBaseConfWithEnv();
+
+  HBaseConfigurationLoader loader;
+  hbase::optional<Configuration> conf = loader.LoadDefaultResources();
+  ASSERT_TRUE(conf) << "No configuration object present.";
+  ASSERT_THROW((*conf).GetBool("bool.exception", false), std::runtime_error);
+}
diff --git a/hbase-native-client/core/hbase_configuration_loader.cc b/hbase-native-client/core/hbase_configuration_loader.cc
new file mode 100644
index 0000000..43f733a
--- /dev/null
+++ b/hbase-native-client/core/hbase_configuration_loader.cc
@@ -0,0 +1,214 @@
+/*
+ * 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/hbase_configuration_loader.h"
+
+#include <boost/foreach.hpp>
+#include <boost/property_tree/ptree.hpp>
+#include <boost/property_tree/xml_parser.hpp>
+#include <glog/logging.h>
+
+namespace hbase {
+
+bool is_valid_bool(const std::string &raw) {
+  if (raw.empty()) {
+    return false;
+  }
+
+  if (!strcasecmp(raw.c_str(), "true")) {
+    return true;
+  }
+  if (!strcasecmp(raw.c_str(), "false")) {
+    return true;
+  }
+  return false;
+}
+
+bool str_to_bool(const std::string &raw) {
+  if (!strcasecmp(raw.c_str(), "true")) {
+    return true;
+  }
+  return false;
+}
+
+HBaseConfigurationLoader::HBaseConfigurationLoader() {}
+
+HBaseConfigurationLoader::~HBaseConfigurationLoader() {}
+
+void HBaseConfigurationLoader::SetDefaultSearchPath() {
+  /*
+   * Try (in order, taking the first valid one):
+   * $HBASE_CONF_DIR
+   * /etc/hbase/conf
+   *
+   */
+  const char *hadoop_conf_dir_env = getenv("HBASE_CONF");
+  if (hadoop_conf_dir_env) {
+    AddToSearchPath(hadoop_conf_dir_env);
+  } else {
+    AddToSearchPath(kHBaseDefauktConfPath);
+  }
+}
+
+void HBaseConfigurationLoader::ClearSearchPath() { search_paths_.clear(); }
+
+void HBaseConfigurationLoader::SetSearchPath(const std::string &search_path) {
+  search_paths_.clear();
+
+  std::vector<std::string> paths;
+  std::string::size_type start = 0;
+  std::string::size_type end = search_path.find(kSearchPathSeparator);
+
+  while (end != std::string::npos) {
+    paths.push_back(search_path.substr(start, end - start));
+    start = ++end;
+    end = search_path.find(kSearchPathSeparator, start);
+  }
+  paths.push_back(search_path.substr(start, search_path.length()));
+
+  for (auto path : paths) {
+    AddToSearchPath(path);
+  }
+}
+
+void HBaseConfigurationLoader::AddToSearchPath(const std::string &search_path) {
+  if (search_path.empty()) return;
+
+  std::string path_to_add(search_path);
+  if (search_path.back() != kFileSeparator) {
+    path_to_add += kFileSeparator;
+  }
+  if (std::find(search_paths_.begin(), search_paths_.end(), path_to_add) ==
+      search_paths_.end())
+    search_paths_.push_back(path_to_add);
+}
+
+void HBaseConfigurationLoader::AddDefaultResources() {
+  resources_.push_back(kHBaseDefaultXml);
+  resources_.push_back(kHBaseSiteXml);
+}
+
+void HBaseConfigurationLoader::AddResources(const std::string &filename) {
+  if (std::find(resources_.begin(), resources_.end(), filename) ==
+      resources_.end())
+    resources_.push_back(filename);
+}
+
+optional<Configuration> HBaseConfigurationLoader::LoadDefaultResources() {
+  SetDefaultSearchPath();
+  AddDefaultResources();
+  ConfigMap conf_property;
+  bool success = false;
+  for (auto dir : search_paths_) {
+    for (auto file : resources_) {
+      std::string config_file = dir + file;
+      std::ifstream stream(config_file);
+      if (stream.is_open()) {
+        success |= LoadProperties(config_file, conf_property);
+      } else {
+        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
+      }
+    }
+  }
+  if (success) {
+    return std::experimental::make_optional<Configuration>(conf_property);
+  } else {
+    return optional<Configuration>();
+  }
+}
+
+optional<Configuration> HBaseConfigurationLoader::LoadResources(
+    const std::string &search_path, const std::vector<std::string> &resources) {
+  SetSearchPath(search_path);
+  for (const auto &resource : resources) AddResources(resource);
+  ConfigMap conf_property;
+  bool success = false;
+  for (auto dir : search_paths_) {
+    for (auto file : resources_) {
+      std::string config_file = dir + file;
+      std::ifstream stream(config_file);
+      if (stream.is_open()) {
+        success |= LoadProperties(config_file, conf_property);
+      } else {
+        DLOG(WARNING) << "Unable to open file[" << config_file << "]";
+      }
+    }
+  }
+  if (success) {
+    return std::experimental::make_optional<Configuration>(conf_property);
+  } else {
+    return optional<Configuration>();
+  }
+}
+
+bool HBaseConfigurationLoader::LoadProperties(const std::string &file,
+                                              ConfigMap &property_map) {
+  // Create empty property tree object
+  using boost::property_tree::ptree;
+  ptree pt;
+  try {
+    // Load XML file and put contents in a property tree.
+    // If read fails, throw exception.
+    read_xml(file, pt);
+
+    // If configuration key is not found exception is thrown
+    std::string configuration = pt.get<std::string>("configuration");
+
+    // Iterate over configuration section.
+    // Store all found properties in ConfigMap
+    BOOST_FOREACH(ptree::value_type & v, pt.get_child("configuration")) {
+      if ("property" == v.first) {
+        std::string name_node = v.second.get<std::string>("name");
+        std::string value_node = v.second.get<std::string>("value");
+        if ((name_node.size() > 0) && (value_node.size() > 0)) {
+          boost::optional<std::string> final_node =
+              v.second.get_optional<std::string>("final");
+          UpdateMapWithValue(property_map, name_node, value_node, final_node);
+        }
+      }
+    }
+  }
+  catch (std::exception &ex) {
+    DLOG(WARNING) << "Exception in parsing file [" << file << "]:[" << ex.what()
+                  << "]";
+    return false;
+  }
+  return true;
+}
+
+bool HBaseConfigurationLoader::UpdateMapWithValue(
+    ConfigMap &map, const std::string &key, const std::string &value,
+    boost::optional<std::string> final_text) {
+  auto map_value = map.find(key);
+  if (map_value != map.end() && map_value->second.final) {
+    return false;
+  }
+
+  bool final_value = false;
+  if (nullptr != final_text.get_ptr()) {
+    if (is_valid_bool(final_text.get())) {
+      final_value = str_to_bool(final_text.get());
+    }
+  }
+
+  map[key].value = value;
+  map[key].final = final_value;
+  return true;
+}
+} /* namespace hbase */
diff --git a/hbase-native-client/core/hbase_configuration_loader.h b/hbase-native-client/core/hbase_configuration_loader.h
new file mode 100644
index 0000000..54d90a5
--- /dev/null
+++ b/hbase-native-client/core/hbase_configuration_loader.h
@@ -0,0 +1,146 @@
+/*
+ * 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 <experimental/optional>
+
+#include "core/configuration.h"
+
+namespace hbase {
+
+template <class T>
+using optional = std::experimental::optional<T>;
+
+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 */