Browse Source

HDFS-9537. libhdfs++ implement HDFSConfiguration class. Contributed by Bob Hansen

James 9 years ago
parent
commit
567c3ed44f
14 changed files with 478 additions and 151 deletions
  1. 5 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
  3. 0 98
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc
  4. 20 23
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.h
  5. 121 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc
  6. 63 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.h
  7. 56 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader_impl.h
  8. 49 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.cc
  9. 55 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h
  10. 10 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
  11. 4 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
  12. 25 24
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc
  13. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h
  14. 67 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h

@@ -29,23 +29,27 @@ struct Options {
    * Default: 30000
    **/
   int rpc_timeout;
+  static const int kDefaultRpcTimeout = 30000;
 
   /**
    * Maximum number of retries for RPC operations
    **/
-  const static int kNoRetry = -1;
   int max_rpc_retries;
+  static const int kNoRetry = -1;
+  static const int kDefaultMaxRpcRetries = kNoRetry;
 
   /**
    * Number of ms to wait between retry of RPC operations
    **/
   int rpc_retry_delay_ms;
+  static const int kDefaultRpcRetryDelayMs = 10000;
 
   /**
    * Exclusion time for failed datanodes in milliseconds.
    * Default: 60000
    **/
   unsigned int host_exclusion_duration;
+  static const unsigned int kDefaultHostExclusionDuration = 600000;
 
   Options();
 };

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt

@@ -15,4 +15,4 @@
 # specific language governing permissions and limitations
 # under the License.
 
-add_library(common base64.cc status.cc sasl_digest_md5.cc hdfs_public_api.cc options.cc configuration.cc util.cc retry_policy.cc)
+add_library(common base64.cc status.cc sasl_digest_md5.cc hdfs_public_api.cc options.cc configuration.cc configuration_loader.cc hdfs_configuration.cc util.cc retry_policy.cc)

+ 0 - 98
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc

@@ -45,104 +45,6 @@ namespace hdfs {
  * Configuration class
  */
 
-Configuration::Configuration() {}
-
-bool is_valid_bool(const std::string& raw) {
-  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;
-}
-
-optional<Configuration> Configuration::Load(const std::string& xmlData) {
-  Configuration result;
-  return result.OverlayResourceString(xmlData);
-}
-
-optional<Configuration> Configuration::OverlayResourceString(
-    const std::string& xmlData) const {
-  if (xmlData.size() == 0) {
-    return optional<Configuration>();
-  }
-
-  int length = xmlData.size();
-  std::vector<char> raw_bytes;
-  raw_bytes.reserve(length + 1);
-  std::copy(xmlData.begin(), xmlData.end(), std::back_inserter(raw_bytes));
-  raw_bytes.push_back('\0');
-
-  ConfigMap map(raw_values_);
-  bool success = UpdateMapWithResource(map, raw_bytes);
-
-  if (success) {
-    return optional<Configuration>(Configuration(map));
-  } else {
-    return optional<Configuration>();
-  }
-}
-
-bool Configuration::UpdateMapWithResource(ConfigMap& map,
-                                          std::vector<char>& raw_bytes) {
-  rapidxml::xml_document<> dom;
-  dom.parse<rapidxml::parse_trim_whitespace>(&raw_bytes[0]);
-
-  /* File must contain a single <configuration> stanza */
-  auto config_node = dom.first_node("configuration", 0, false);
-  if (!config_node) {
-    return false;
-  }
-
-  /* Walk all of the <property> nodes, ignoring the rest */
-  for (auto property_node = config_node->first_node("property", 0, false);
-       property_node;
-       property_node = property_node->next_sibling("property", 0, false)) {
-    auto name_node = property_node->first_node("name", 0, false);
-    auto value_node = property_node->first_node("value", 0, false);
-
-    if (name_node && value_node) {
-      auto mapValue = map.find(name_node->value());
-      if (mapValue != map.end() && mapValue->second.final) {
-        continue;
-      }
-
-      map[name_node->value()] = value_node->value();
-      auto final_node = property_node->first_node("final", 0, false);
-      if (final_node && is_valid_bool(final_node->value())) {
-        map[name_node->value()].final = str_to_bool(final_node->value());
-      }
-    }
-
-    auto name_attr = property_node->first_attribute("name", 0, false);
-    auto value_attr = property_node->first_attribute("value", 0, false);
-
-    if (name_attr && value_attr) {
-      auto mapValue = map.find(name_attr->value());
-      if (mapValue != map.end() && mapValue->second.final) {
-        continue;
-      }
-
-      map[name_attr->value()] = value_attr->value();
-      auto final_attr = property_node->first_attribute("final", 0, false);
-      if (final_attr && is_valid_bool(final_attr->value())) {
-        map[name_attr->value()].final = str_to_bool(final_attr->value());
-      }
-    }
-  }
-
-  return true;
-}
-
 optional<std::string> Configuration::Get(const std::string& key) const {
   auto found = raw_values_.find(key);
   if (found != raw_values_.end()) {

+ 20 - 23
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.h

@@ -43,33 +43,30 @@ using optional = std::experimental::optional<T>;
  *  </property>
  * <configuration>
  *
- * This class is not thread-safe.
+ * Configuration objects should be created via the ConfigurationLoader class.
+ * Configuration objects are immutable and can be shared between threads.
+ *
+ * This class is thread-safe.
  */
 class Configuration {
  public:
-  /* Creates a new Configuration from input xml */
-  static optional<Configuration> Load(const std::string &xml_data);
-
-  /* Constructs a configuration with no resources loaded */
-  Configuration();
-
-  /* Loads resources from a file or a stream */
-  optional<Configuration> OverlayResourceString(
-      const std::string &xml_data) const;
-
   // Gets values
-  std::string GetWithDefault(const std::string &key,
-                             const std::string &default_value) const;
+  std::string           GetWithDefault(const std::string &key,
+                                       const std::string &default_value) const;
   optional<std::string> Get(const std::string &key) const;
-  int64_t GetIntWithDefault(const std::string &key, int64_t default_value) const;
-  optional<int64_t> GetInt(const std::string &key) const;
-  double GetDoubleWithDefault(const std::string &key,
-                              double default_value) const;
-  optional<double> GetDouble(const std::string &key) const;
-  bool GetBoolWithDefault(const std::string &key, bool default_value) const;
-  optional<bool> GetBool(const std::string &key) const;
+  int64_t               GetIntWithDefault(const std::string &key,
+                                          int64_t default_value) const;
+  optional<int64_t>     GetInt(const std::string &key) const;
+  double                GetDoubleWithDefault(const std::string &key,
+                                             double default_value) const;
+  optional<double>      GetDouble(const std::string &key) const;
+  bool                  GetBoolWithDefault(const std::string &key,
+                                           bool default_value) const;
+  optional<bool>        GetBool(const std::string &key) const;
+
+protected:
+   friend class ConfigurationLoader;
 
- private:
   /* Transparent data holder for property values */
   struct ConfigData {
     std::string value;
@@ -83,12 +80,12 @@ class Configuration {
   };
   typedef std::map<std::string, ConfigData> ConfigMap;
 
+  Configuration() {};
   Configuration(ConfigMap &src_map) : raw_values_(src_map){};
-  static bool UpdateMapWithResource(ConfigMap &map,
-                                    std::vector<char> &raw_bytes);
 
   const ConfigMap raw_values_;
 };
+
 }
 
 #endif

+ 121 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc

@@ -0,0 +1,121 @@
+/**
+ * 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_loader.h"
+
+#include <strings.h>
+#include <sstream>
+#include <map>
+#include <rapidxml/rapidxml.hpp>
+#include <rapidxml/rapidxml_utils.hpp>
+
+namespace hdfs {
+
+/*
+ * ConfigurationBuilder class
+ */
+
+bool is_valid_bool(const std::string& raw) {
+  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;
+}
+
+
+bool ConfigurationLoader::UpdateMapWithString(ConfigMap & map,
+                                                   const std::string &xml_data) {
+  if (xml_data.size() == 0) {
+    return false;
+  }
+
+  std::vector<char> raw_bytes(xml_data.begin(), xml_data.end());
+  raw_bytes.push_back('\0');
+
+  bool success = UpdateMapWithBytes(map, raw_bytes);
+
+  if (success) {
+    return true;
+  } else {
+    return false;
+  }
+}
+
+bool ConfigurationLoader::UpdateMapWithBytes(ConfigMap& map,
+                                                 std::vector<char>& raw_bytes) {
+  rapidxml::xml_document<> dom;
+  dom.parse<rapidxml::parse_trim_whitespace>(&raw_bytes[0]);
+
+  /* File must contain a single <configuration> stanza */
+  auto config_node = dom.first_node("configuration", 0, false);
+  if (!config_node) {
+    return false;
+  }
+
+  /* Walk all of the <property> nodes, ignoring the rest */
+  for (auto property_node = config_node->first_node("property", 0, false);
+       property_node;
+       property_node = property_node->next_sibling("property", 0, false)) {
+    auto name_node = property_node->first_node("name", 0, false);
+    auto value_node = property_node->first_node("value", 0, false);
+
+    if (name_node && value_node) {
+      auto mapValue = map.find(name_node->value());
+      if (mapValue != map.end() && mapValue->second.final) {
+        continue;
+      }
+
+      map[name_node->value()] = value_node->value();
+      auto final_node = property_node->first_node("final", 0, false);
+      if (final_node && is_valid_bool(final_node->value())) {
+        map[name_node->value()].final = str_to_bool(final_node->value());
+      }
+    }
+
+    auto name_attr = property_node->first_attribute("name", 0, false);
+    auto value_attr = property_node->first_attribute("value", 0, false);
+
+    if (name_attr && value_attr) {
+      auto mapValue = map.find(name_attr->value());
+      if (mapValue != map.end() && mapValue->second.final) {
+        continue;
+      }
+
+      map[name_attr->value()] = value_attr->value();
+      auto final_attr = property_node->first_attribute("final", 0, false);
+      if (final_attr && is_valid_bool(final_attr->value())) {
+        map[name_attr->value()].final = str_to_bool(final_attr->value());
+      }
+    }
+  }
+
+  return true;
+}
+
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.h

@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+
+#ifndef COMMON_CONFIGURATION_BUILDER_H_
+#define COMMON_CONFIGURATION_BUILDER_H_
+
+#include "configuration.h"
+
+namespace hdfs {
+
+
+class ConfigurationLoader {
+public:
+  // Creates a new, empty Configuration object
+  //    T must be Configuration or a subclass
+  template<class T>
+  T           New();
+
+  // Loads Configuration XML contained in a string and returns a parsed
+  //    Configuration object
+  //    T must be Configuration or a subclass
+  template<class T>
+  optional<T> Load(const std::string &xml_data);
+
+  // Loads Configuration XML contained in a string and produces a new copy that
+  //    is the union of the src and xml_data
+  //    Any parameters from src will be overwritten by the xml_data unless they
+  //    are marked as "final" in src.
+  //    T must be Configuration or a subclass
+  template<class T>
+  optional<T> OverlayResourceString(const T &src, const std::string &xml_data) const;
+
+protected:
+  using ConfigMap = Configuration::ConfigMap;
+
+  // Updates the src map with data from the XML
+  static bool UpdateMapWithString( Configuration::ConfigMap & src,
+                                   const std::string &xml_data);
+  // Updates the src map with data from the XML
+  static bool UpdateMapWithBytes(Configuration::ConfigMap &map,
+                                 std::vector<char> &raw_bytes);
+};
+
+}
+
+#include "configuration_loader_impl.h"
+
+#endif

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader_impl.h

@@ -0,0 +1,56 @@
+/**
+ * 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.
+ */
+
+#ifndef COMMON_CONFIGURATION_BUILDER_IMPL_H_
+#define COMMON_CONFIGURATION_BUILDER_IMPL_H_
+
+namespace hdfs {
+
+
+template<class T>
+T ConfigurationLoader::New() {
+  return T();
+}
+
+template<class T>
+optional<T> ConfigurationLoader::Load(const std::string &xml_data) {
+  return OverlayResourceString<T>(T(), xml_data);
+}
+
+template<class T>
+optional<T> ConfigurationLoader::OverlayResourceString(const T& src, const std::string &xml_data) const {
+  if (xml_data.size() == 0) {
+    return optional<T>();
+  }
+
+  std::vector<char> raw_bytes(xml_data.begin(), xml_data.end());
+  raw_bytes.push_back('\0');
+
+  ConfigMap map(src.raw_values_);
+  bool success = UpdateMapWithBytes(map, raw_bytes);
+
+  if (success) {
+    return std::experimental::make_optional<T>(map);
+  } else {
+    return optional<T>();
+  }
+}
+
+}
+
+#endif

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.cc

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "common/hdfs_configuration.h"
+
+namespace hdfs {
+
+// Constructs a configuration with no search path and no resources loaded
+HdfsConfiguration::HdfsConfiguration() : Configuration() {}
+
+// Constructs a configuration with a copy of the input data
+HdfsConfiguration::HdfsConfiguration(ConfigMap &src_map) : Configuration(src_map) {}
+
+
+// Sets a value iff the optional<T> has a value
+template <class T, class U>
+void OptionalSet(T& target, optional<U> value) {
+  if (value)
+    target = *value;
+}
+
+// Interprets the resources to build an Options object
+Options HdfsConfiguration::GetOptions() {
+  Options result;
+
+  OptionalSet(result.rpc_timeout, GetInt(kDfsClientSocketTimeoutKey));
+  OptionalSet(result.max_rpc_retries, GetInt(kIpcClientConnectMaxRetriesKey));
+  OptionalSet(result.rpc_retry_delay_ms, GetInt(kIpcClientConnectRetryIntervalKey));
+
+  return result;
+}
+
+
+}

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h

@@ -0,0 +1,55 @@
+/**
+ * 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.
+ */
+
+#ifndef COMMON_HDFS_CONFIGURATION_H_
+#define COMMON_HDFS_CONFIGURATION_H_
+
+#include "common/configuration.h"
+#include "libhdfspp/options.h"
+
+#include <string>
+#include <map>
+#include <vector>
+#include <set>
+#include <istream>
+#include <stdint.h>
+
+namespace hdfs {
+
+class HdfsConfiguration : public Configuration {
+  public:
+    // Interprets the resources to build an Options object
+    Options GetOptions();
+
+    // Keys to look for in the configuration file
+    static constexpr const char * kDfsClientSocketTimeoutKey = "dfs.client.socket-timeout";
+    static constexpr const char * kIpcClientConnectMaxRetriesKey = "ipc.client.connect.max.retries";
+    static constexpr const char * kIpcClientConnectRetryIntervalKey = "ipc.client.connect.retry.interval";
+private:
+    friend class ConfigurationLoader;
+
+    // Constructs a configuration with no search path and no resources loaded
+    HdfsConfiguration();
+
+    // Constructs a configuration with some static data
+    HdfsConfiguration(ConfigMap &src_map);
+};
+
+}
+
+#endif

+ 10 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc

@@ -20,6 +20,14 @@
 
 namespace hdfs {
 
-Options::Options() : rpc_timeout(30000), max_rpc_retries(kNoRetry),
-                     rpc_retry_delay_ms(10000), host_exclusion_duration(600000) {}
+// The linker needs a place to put all of those constants
+const int Options::kDefaultRpcTimeout;
+const int Options::kNoRetry;
+const int Options::kDefaultMaxRpcRetries;
+const int Options::kDefaultRpcRetryDelayMs;
+const unsigned int Options::kDefaultHostExclusionDuration;
+
+Options::Options() : rpc_timeout(kDefaultRpcTimeout), max_rpc_retries(kDefaultMaxRpcRetries),
+                     rpc_retry_delay_ms(kDefaultRpcRetryDelayMs),
+                     host_exclusion_duration(kDefaultHostExclusionDuration) {}
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt

@@ -72,6 +72,10 @@ add_executable(configuration_test configuration_test.cc)
 target_link_libraries(configuration_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_test(configuration configuration_test)
 
+add_executable(hdfs_configuration_test hdfs_configuration_test.cc)
+target_link_libraries(hdfs_configuration_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
+add_test(hdfs_configuration hdfs_configuration_test)
+
 build_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static expect.c test_libhdfs_threaded.c ${OS_DIR}/thread.c)
 link_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static fs reader rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY})
 add_libhdfs_test(libhdfs_threaded hdfspp_test_shim_static)

+ 25 - 24
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc

@@ -18,6 +18,7 @@
 
 #include "configuration_test.h"
 #include "common/configuration.h"
+#include "common/configuration_loader.h"
 #include <gmock/gmock.h>
 #include <cstdio>
 #include <fstream>
@@ -32,21 +33,21 @@ TEST(ConfigurationTest, TestDegenerateInputs) {
   /* Completely empty stream */
   {
     std::stringstream stream;
-    optional<Configuration> config = Configuration::Load("");
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>("");
     EXPECT_FALSE(config && "Empty stream");
   }
 
   /* No values */
   {
     std::string data = "<configuration></configuration>";
-    optional<Configuration> config = Configuration::Load(data);
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(data);
     EXPECT_TRUE(config && "Blank config");
   }
 
   /* Extraneous values */
   {
     std::string data = "<configuration><spam></spam></configuration>";
-    optional<Configuration> config = Configuration::Load(data);
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(data);
     EXPECT_TRUE(config && "Extraneous values");
   }
 }
@@ -56,7 +57,7 @@ TEST(ConfigurationTest, TestBasicOperations) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "value1");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
   }
@@ -73,7 +74,7 @@ TEST(ConfigurationTest, TestBasicOperations) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "value1");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     optional<std::string> value = config->Get("key1");
     EXPECT_TRUE((bool)value);
@@ -87,7 +88,7 @@ TEST(ConfigurationTest, TestCompactValues) {
     std::stringstream stream;
     stream << "<configuration><property name=\"key1\" "
               "value=\"value1\"/></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Compact value parse");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
   }
@@ -98,14 +99,14 @@ TEST(ConfigurationTest, TestMultipleResources) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "value1");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key2", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value1", config2->GetWithDefault("key1", ""));
     EXPECT_EQ("value2", config2->GetWithDefault("key2", ""));
@@ -119,7 +120,7 @@ TEST(ConfigurationTest, TestStringResource) {
     simpleConfigStream(stream, "key1", "value1");
     std::string str = stream.str();
 
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
   }
@@ -131,14 +132,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property><name>key1</name><value>value1</"
               "value><final>false</final></property></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value2", config2->GetWithDefault("key1", ""));
   }
@@ -147,14 +148,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property><name>key1</name><value>value1</"
               "value><final>true</final></property></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value1", config2->GetWithDefault("key1", ""));
   }
@@ -163,14 +164,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property name=\"key1\" value=\"value1\" "
               "final=\"false\"/></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value2", config2->GetWithDefault("key1", ""));
   }
@@ -179,14 +180,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property name=\"key1\" value=\"value1\" "
               "final=\"true\"/></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value1", config2->GetWithDefault("key1", ""));
   }
@@ -195,14 +196,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property><name>key1</name><value>value1</"
               "value><final>spam</final></property></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value2", config2->GetWithDefault("key1", ""));
   }
@@ -211,14 +212,14 @@ TEST(ConfigurationTest, TestFinal) {
     std::stringstream stream;
     stream << "<configuration><property><name>key1</name><value>value1</"
               "value><final></final></property></configuration>";
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     std::stringstream stream2;
     simpleConfigStream(stream2, "key1", "value2");
     optional<Configuration> config2 =
-        config->OverlayResourceString(stream2.str());
+        ConfigurationLoader().OverlayResourceString(config.value(), stream2.str());
     EXPECT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value2", config2->GetWithDefault("key1", ""));
   }
@@ -228,7 +229,7 @@ TEST(ConfigurationTest, TestIntConversions) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "1");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     optional<int64_t> value = config->GetInt("key1");
     EXPECT_TRUE((bool)value);
@@ -267,7 +268,7 @@ TEST(ConfigurationTest, TestDoubleConversions) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "1");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     optional<double> value = config->GetDouble("key1");
     EXPECT_TRUE((bool)value);
@@ -310,7 +311,7 @@ TEST(ConfigurationTest, TestBoolConversions) {
   {
     std::stringstream stream;
     simpleConfigStream(stream, "key1", "true");
-    optional<Configuration> config = Configuration::Load(stream.str());
+    optional<Configuration> config = ConfigurationLoader().Load<Configuration>(stream.str());
     EXPECT_TRUE(config && "Parse single value");
     optional<bool> value = config->GetBool("key1");
     EXPECT_TRUE((bool)value);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h

@@ -19,6 +19,7 @@
 #define TESTS_CONFIGURATION_H_
 
 #include "common/configuration.h"
+#include "common/configuration_loader.h"
 #include <cstdio>
 #include <fstream>
 #include <istream>
@@ -50,10 +51,9 @@ void simpleConfigStream(std::stringstream& out, Args... args) {
 
 template <typename... Args>
 optional<Configuration> simpleConfig(Args... args) {
-  Configuration result;
   std::stringstream stream;
   simpleConfigStream(stream, args...);
-  optional<Configuration> parse = result.Load(stream.str());
+  optional<Configuration> parse = ConfigurationLoader().Load<Configuration>(stream.str());
   EXPECT_TRUE((bool)parse);
 
   return parse;

+ 67 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc

@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "common/hdfs_configuration.h"
+#include "configuration_test.h"
+#include <gmock/gmock.h>
+
+using ::testing::_;
+
+using namespace hdfs;
+
+namespace hdfs
+{
+
+TEST(HdfsConfigurationTest, TestDefaultOptions)
+{
+  // Completely empty stream
+  {
+    HdfsConfiguration empty_config = ConfigurationLoader().New<HdfsConfiguration>();
+    Options options = empty_config.GetOptions();
+    EXPECT_EQ(Options::kDefaultRpcTimeout, options.rpc_timeout);
+  }
+}
+
+TEST(HdfsConfigurationTest, TestSetOptions)
+{
+  // Completely empty stream
+  {
+    std::stringstream stream;
+    simpleConfigStream(stream, HdfsConfiguration::kDfsClientSocketTimeoutKey, 100,
+                               HdfsConfiguration::kIpcClientConnectMaxRetriesKey, 101,
+                               HdfsConfiguration::kIpcClientConnectRetryIntervalKey, 102);
+
+    optional<HdfsConfiguration> config = ConfigurationLoader().Load<HdfsConfiguration>(stream.str());
+    EXPECT_TRUE(config && "Read stream");
+    Options options = config->GetOptions();
+
+    EXPECT_EQ(100, options.rpc_timeout);
+    EXPECT_EQ(101, options.max_rpc_retries);
+    EXPECT_EQ(102, options.rpc_retry_delay_ms);
+  }
+}
+
+int main(int argc, char *argv[])
+{
+  // The following line must be executed to initialize Google Mock
+  // (and Google Test) before running the tests.
+  ::testing::InitGoogleMock(&argc, argv);
+  return RUN_ALL_TESTS();
+}
+
+}