소스 검색

HDFS-9538. libhdfs: load configuration from files. Contributed by Bob Hansen.

James 9 년 전
부모
커밋
6c80d02a03

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

@@ -44,6 +44,12 @@ namespace hdfs {
 /*
  * Configuration class
  */
+std::vector<std::string> Configuration::GetDefaultFilenames() {
+  auto result = std::vector<std::string>();
+  result.push_back("core-site.xml");
+  return result;
+}
+
 
 optional<std::string> Configuration::Get(const std::string& key) const {
   auto found = raw_values_.find(key);

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

@@ -83,6 +83,8 @@ protected:
   Configuration() {};
   Configuration(ConfigMap &src_map) : raw_values_(src_map){};
 
+  static std::vector<std::string> GetDefaultFilenames();
+
   const ConfigMap raw_values_;
 };
 

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

@@ -18,18 +18,28 @@
 
 #include "configuration_loader.h"
 
+#include <fstream>
 #include <strings.h>
 #include <sstream>
 #include <map>
+#include <sys/stat.h>
 #include <rapidxml/rapidxml.hpp>
 #include <rapidxml/rapidxml_utils.hpp>
 
 namespace hdfs {
 
 /*
- * ConfigurationBuilder class
+ * ConfigurationLoader class
  */
 
+#if defined(WIN32) || defined(_WIN32)
+static const char kFileSeparator = '\\';
+#else
+static const char kFileSeparator = '/';
+#endif
+
+static const char kSearchPathSeparator = ':';
+
 bool is_valid_bool(const std::string& raw) {
   if (!strcasecmp(raw.c_str(), "true")) {
     return true;
@@ -48,6 +58,108 @@ bool str_to_bool(const std::string& raw) {
   return false;
 }
 
+void ConfigurationLoader::SetDefaultSearchPath() {
+  //TODO: Use HADOOP_CONF_DIR when we get environment subs with HDFS-9385
+  AddToSearchPath("./");
+  AddToSearchPath("/etc/hadoop/");
+}
+
+void ConfigurationLoader::ClearSearchPath()
+{
+  search_path_.clear();
+}
+
+void ConfigurationLoader::SetSearchPath(const std::string & searchPath)
+{
+  search_path_.clear();
+
+  std::vector<std::string> paths;
+  std::string::size_type start = 0;
+  std::string::size_type end = searchPath.find(kSearchPathSeparator);
+
+  while (end != std::string::npos) {
+     paths.push_back(searchPath.substr(start, end-start));
+     start = ++end;
+     end = searchPath.find(kSearchPathSeparator, start);
+  }
+  paths.push_back(searchPath.substr(start, searchPath.length()));
+
+  for (auto path: paths) {
+    AddToSearchPath(path);
+  }
+
+}
+
+void ConfigurationLoader::AddToSearchPath(const std::string & searchPath)
+{
+  if (searchPath.empty())
+    return;
+
+  if (searchPath.back() != kFileSeparator) {
+    std::string pathWithSlash(searchPath);
+    pathWithSlash += kFileSeparator;
+    search_path_.push_back(pathWithSlash);
+  } else {
+    search_path_.push_back(searchPath);
+  }
+}
+
+std::string ConfigurationLoader::GetSearchPath()
+{
+  std::stringstream result;
+  bool first = true;
+  for(std::string item: search_path_) {
+    if (!first) {
+      result << kSearchPathSeparator;
+    }
+
+    result << item;
+    first = false;
+  }
+
+  return result.str();
+}
+
+bool ConfigurationLoader::UpdateMapWithFile(ConfigMap & map, const std::string & path) const
+{
+  if (path.front() == kFileSeparator) { // Absolute path
+    std::ifstream stream(path, std::ifstream::in);
+    if ( stream.is_open() ) {
+      return UpdateMapWithStream(map, stream);
+    } else {
+      return false;
+    }
+  } else { // Use search path
+    for(auto dir: search_path_) {
+      std::ifstream stream(dir + path);
+      if ( stream.is_open() ) {
+        if (UpdateMapWithStream(map, stream))
+          return true;
+      }
+    }
+  }
+
+  return false;
+}
+
+bool ConfigurationLoader::UpdateMapWithStream(ConfigMap & map,
+                                              std::istream & stream) {
+  std::streampos start = stream.tellg();
+  stream.seekg(0, std::ios::end);
+  std::streampos end = stream.tellg();
+  stream.seekg(start, std::ios::beg);
+
+  int length = end - start;
+
+  if (length <= 0 || start == -1 || end == -1)
+    return false;
+
+  std::vector<char> raw_bytes((int64_t)length + 1);
+  stream.read(&raw_bytes[0], length);
+  raw_bytes[length] = 0;
+
+  return UpdateMapWithBytes(map, raw_bytes);
+}
 
 bool ConfigurationLoader::UpdateMapWithString(ConfigMap & map,
                                                    const std::string &xml_data) {

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

@@ -31,11 +31,22 @@ public:
   template<class T>
   T           New();
 
-  // Loads Configuration XML contained in a string and returns a parsed
-  //    Configuration object
+  /****************************************************************************
+   *                    LOADING CONFIG FILES
+   ***************************************************************************/
+
+  // Loads Configuration XML contained in a string/stream/file and returns a parsed
+  //    Configuration object.
   //    T must be Configuration or a subclass
   template<class T>
   optional<T> Load(const std::string &xml_data);
+  // Streams must be seekable
+  template<class T>
+  optional<T> LoadFromStream(std::istream & stream);
+  // The ConfigurationBuilder's search path will be searched for the filename
+  //    unless it is an absolute path
+  template<class T>
+  optional<T> LoadFromFile(const std::string &filename);
 
   // Loads Configuration XML contained in a string and produces a new copy that
   //    is the union of the src and xml_data
@@ -44,16 +55,56 @@ public:
   //    T must be Configuration or a subclass
   template<class T>
   optional<T> OverlayResourceString(const T &src, const std::string &xml_data) const;
+  // Streams must be seekable
+  template<class T>
+  optional<T> OverlayResourceStream(const T &src, std::istream &stream) const;
+  //    The ConfigurationBuilder's search path will be searched for the filename
+  //       unless it is an absolute path
+  template<class T>
+  optional<T> OverlayResourceFile(const T &src, const std::string &path) const;
+
+  // Returns an instance of the Configuration with all of the default resource
+  //    files loaded.
+  //    T must be Configuration or a subclass
+  template<class T>
+  optional<T> LoadDefaultResources();
+
+
+  /****************************************************************************
+   *                    SEARCH PATH METHODS
+   ***************************************************************************/
+
+  // Sets the search path to the default search path (namely, ".:/etc/hadoop")
+  void SetDefaultSearchPath();
+
+  // Clears out the search path
+  void ClearSearchPath();
+  // Sets the search path to ":"-delimited paths
+  void SetSearchPath(const std::string & searchPath);
+  // Adds an element to the search path
+  void AddToSearchPath(const std::string & searchPath);
+  // Returns the search path in ":"-delmited form
+  std::string GetSearchPath();
 
 protected:
   using ConfigMap = Configuration::ConfigMap;
 
+  // Updates the src map with data from the XML in the path
+  //   The search path will be searched for the filename
+  bool UpdateMapWithFile(ConfigMap & map, const std::string & path) const;
+
+  // Updates the src map with data from the XML in the stream
+  //   The stream must be seekable
+  static bool UpdateMapWithStream(ConfigMap & map,
+                                  std::istream & stream);
   // Updates the src map with data from the XML
-  static bool UpdateMapWithString( Configuration::ConfigMap & src,
-                                   const std::string &xml_data);
+  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);
+
+  std::vector<std::string> search_path_;
 };
 
 }

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

@@ -31,6 +31,39 @@ template<class T>
 optional<T> ConfigurationLoader::Load(const std::string &xml_data) {
   return OverlayResourceString<T>(T(), xml_data);
 }
+template<class T>
+optional<T> ConfigurationLoader::LoadFromStream(std::istream &stream) {
+  return OverlayResourceStream<T>(T(), stream);
+}
+template<class T>
+optional<T> ConfigurationLoader::LoadFromFile(const std::string &path) {
+  return OverlayResourceFile<T>(T(), path);
+}
+
+
+template<class T>
+optional<T> ConfigurationLoader::OverlayResourceFile(const T& src, const std::string &path) const {
+  ConfigMap map(src.raw_values_);
+  bool success = UpdateMapWithFile(map, path);
+
+  if (success) {
+    return std::experimental::make_optional<T>(map);
+  } else {
+    return optional<T>();
+  }
+}
+
+template<class T>
+optional<T> ConfigurationLoader::OverlayResourceStream(const T& src, std::istream & stream) const {
+  ConfigMap map(src.raw_values_);
+  bool success = UpdateMapWithStream(map, stream);
+
+  if (success) {
+    return std::experimental::make_optional<T>(map);
+  } else {
+    return optional<T>();
+  }
+}
 
 template<class T>
 optional<T> ConfigurationLoader::OverlayResourceString(const T& src, const std::string &xml_data) const {
@@ -51,6 +84,27 @@ optional<T> ConfigurationLoader::OverlayResourceString(const T& src, const std::
   }
 }
 
+template <class T>
+optional<T> ConfigurationLoader::LoadDefaultResources() {
+  std::vector<std::string> default_filenames = T::GetDefaultFilenames();
+
+  ConfigMap result;
+  bool success = true;
+
+  for (auto fn: default_filenames) {
+    success &= UpdateMapWithFile(result, fn);
+    if (!success)
+      break;
+  }
+
+  if (success) {
+    return std::experimental::make_optional<T>(result);
+  } else {
+    return optional<T>();
+  }
+}
+
+
 }
 
 #endif

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

@@ -26,6 +26,11 @@ HdfsConfiguration::HdfsConfiguration() : Configuration() {}
 // Constructs a configuration with a copy of the input data
 HdfsConfiguration::HdfsConfiguration(ConfigMap &src_map) : Configuration(src_map) {}
 
+std::vector<std::string> HdfsConfiguration::GetDefaultFilenames() {
+  auto result = Configuration::GetDefaultFilenames();
+  result.push_back("hdfs-site.xml");
+  return result;
+}
 
 // Sets a value iff the optional<T> has a value
 template <class T, class U>

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

@@ -48,6 +48,8 @@ private:
 
     // Constructs a configuration with some static data
     HdfsConfiguration(ConfigMap &src_map);
+
+    static std::vector<std::string> GetDefaultFilenames();
 };
 
 }

+ 91 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc

@@ -224,6 +224,97 @@ TEST(ConfigurationTest, TestFinal) {
     EXPECT_EQ("value2", config2->GetWithDefault("key1", ""));
   }
 }
+
+TEST(ConfigurationTest, TestFileReads)
+{
+  // Single stream
+  {
+    TempFile tempFile;
+    writeSimpleConfig(tempFile.filename, "key1", "value1");
+
+    optional<Configuration> config = ConfigurationLoader().LoadFromFile<Configuration>(tempFile.filename);
+    EXPECT_TRUE(config && "Parse first stream");
+    EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
+  }
+
+  // Multiple files
+  {
+    TempFile tempFile;
+    writeSimpleConfig(tempFile.filename, "key1", "value1");
+
+    ConfigurationLoader loader;
+    optional<Configuration> config = loader.LoadFromFile<Configuration>(tempFile.filename);
+    ASSERT_TRUE(config && "Parse first stream");
+    EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
+
+    TempFile tempFile2;
+    writeSimpleConfig(tempFile2.filename, "key2", "value2");
+    optional<Configuration> config2 = loader.OverlayResourceFile(*config, tempFile2.filename);
+    ASSERT_TRUE(config2 && "Parse second stream");
+    EXPECT_EQ("value1", config2->GetWithDefault("key1", ""));
+    EXPECT_EQ("value2", config2->GetWithDefault("key2", ""));
+  }
+
+  // Try to add a directory
+  {
+    TempDir tempDir;
+
+    optional<Configuration> config = ConfigurationLoader().LoadFromFile<Configuration>(tempDir.path);
+    EXPECT_FALSE(config && "Add directory as file resource");
+  }
+
+
+  // Search path splitting
+  {
+    ConfigurationLoader loader;
+    loader.SetSearchPath("foo:/bar:baz/:/fioux/:/bar/bar/bong");
+
+    // Paths will have / appended to them if not already present
+    EXPECT_EQ("foo/:/bar/:baz/:/fioux/:/bar/bar/bong/", loader.GetSearchPath());
+  }
+
+  // Search path
+  {
+    TempDir tempDir1;
+    TempFile tempFile1(tempDir1.path + "/file1.xml");
+    writeSimpleConfig(tempFile1.filename, "key1", "value1");
+    TempDir tempDir2;
+    TempFile tempFile2(tempDir2.path + "/file2.xml");
+    writeSimpleConfig(tempFile2.filename, "key2", "value2");
+    TempDir tempDir3;
+    TempFile tempFile3(tempDir3.path + "/file3.xml");
+    writeSimpleConfig(tempFile3.filename, "key3", "value3");
+
+    ConfigurationLoader loader;
+    loader.SetSearchPath(tempDir1.path + ":" + tempDir2.path + ":" + tempDir3.path);
+    optional<Configuration> config1 = loader.LoadFromFile<Configuration>("file1.xml");
+    EXPECT_TRUE(config1 && "Parse first stream");
+    optional<Configuration> config2 = loader.OverlayResourceFile(*config1, "file2.xml");
+    EXPECT_TRUE(config2 && "Parse second stream");
+    optional<Configuration> config3 = loader.OverlayResourceFile(*config2, "file3.xml");
+    EXPECT_TRUE(config3 && "Parse third stream");
+    EXPECT_EQ("value1", config3->GetWithDefault("key1", ""));
+    EXPECT_EQ("value2", config3->GetWithDefault("key2", ""));
+    EXPECT_EQ("value3", config3->GetWithDefault("key3", ""));
+  }
+}
+
+TEST(ConfigurationTest, TestDefaultConfigs) {
+  // Search path
+  {
+    TempDir tempDir;
+    TempFile coreSite(tempDir.path + "/core-site.xml");
+    writeSimpleConfig(coreSite.filename, "key1", "value1");
+
+    ConfigurationLoader loader;
+    loader.SetSearchPath(tempDir.path);
+
+    optional<Configuration> config = loader.LoadDefaultResources<Configuration>();
+    EXPECT_TRUE(config && "Parse streams");
+    EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
+  }
+}
+
 TEST(ConfigurationTest, TestIntConversions) {
   /* No defaults */
   {

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

@@ -23,6 +23,7 @@
 #include <cstdio>
 #include <fstream>
 #include <istream>
+#include <ftw.h>
 #include <gmock/gmock.h>
 
 namespace hdfs {
@@ -68,6 +69,54 @@ void writeSimpleConfig(const std::string& filename, Args... args) {
   out.open(filename);
   out << stream.rdbuf();
 }
+
+// TempDir: is deleted on destruction
+class TempFile {
+public:
+  std::string filename;
+  char        fn_buffer[128];
+  int         tempFileHandle;
+  TempFile() : tempFileHandle(-1) {
+    strncpy(fn_buffer, "/tmp/test_XXXXXXXXXX", sizeof(fn_buffer));
+    tempFileHandle = mkstemp(fn_buffer);
+    EXPECT_NE(-1, tempFileHandle);
+    filename = fn_buffer;
+  }
+  TempFile(const std::string & fn) : filename(fn), tempFileHandle(-1) {
+    strncpy(fn_buffer, fn.c_str(), sizeof(fn_buffer));
+  }
+  ~TempFile() { close(tempFileHandle); unlink(fn_buffer); }
+};
+
+
+// Callback to remove a directory in the nftw visitor
+int nftw_remove(const char *fpath, const struct stat *sb, int typeflag, struct FTW *ftwbuf)
+{
+  (void)sb; (void)typeflag; (void)ftwbuf;
+
+  int rv = remove(fpath);
+  EXPECT_EQ(0, rv);
+  return rv;
+}
+
+// TempDir: is created in ctor and recursively deletes in dtor
+class TempDir {
+public:
+  std::string path;
+  TempDir() {
+    char        fn_buffer[128];
+    strncpy(fn_buffer, "/tmp/test_dir_XXXXXXXXXX", sizeof(fn_buffer));
+    const char * returned_path = mkdtemp(fn_buffer);
+    EXPECT_NE(nullptr, returned_path);
+    path = returned_path;
+  }
+  ~TempDir() {
+    if(!path.empty())
+      nftw(path.c_str(), nftw_remove, 64, FTW_DEPTH | FTW_PHYS);
+  }
+};
+
+
 }
 
 #endif

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

@@ -56,6 +56,25 @@ TEST(HdfsConfigurationTest, TestSetOptions)
   }
 }
 
+TEST(HdfsConfigurationTest, TestDefaultConfigs) {
+  // Search path
+  {
+    TempDir tempDir;
+    TempFile coreSite(tempDir.path + "/core-site.xml");
+    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    TempFile hdfsSite(tempDir.path + "/hdfs-site.xml");
+    writeSimpleConfig(hdfsSite.filename, "key2", "value2");
+
+    ConfigurationLoader loader;
+    loader.SetSearchPath(tempDir.path);
+
+    optional<HdfsConfiguration> config = loader.LoadDefaultResources<HdfsConfiguration>();
+    EXPECT_TRUE(config && "Parse streams");
+    EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
+    EXPECT_EQ("value2", config->GetWithDefault("key2", ""));
+  }
+}
+
 int main(int argc, char *argv[])
 {
   // The following line must be executed to initialize Google Mock