Browse Source

HDFS-6994. use libexpat instead of libxml2 for libhdfs3 (cmccabe)

Colin Patrick Mccabe 10 năm trước cách đây
mục cha
commit
411c31faba
19 tập tin đã thay đổi với 630 bổ sung781 xóa
  1. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/bootstrap
  3. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
  4. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.cc
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.h
  6. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
  7. 108 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc
  8. 40 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h
  9. 106 122
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.cc
  10. 10 54
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.h
  11. 0 291
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.cc
  12. 0 165
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.h
  13. 27 11
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.cc
  14. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.h
  15. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
  16. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StringUtil.h
  17. 269 91
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.cc
  18. 23 25
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.h
  19. 14 10
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt

@@ -37,6 +37,7 @@ FIND_PACKAGE(LibXml2 REQUIRED)
 FIND_PACKAGE(Protobuf REQUIRED)
 FIND_PACKAGE(KERBEROS REQUIRED)
 FIND_PACKAGE(GSasl REQUIRED)
+FIND_PACKAGE(EXPAT REQUIRED)
 
 ADD_SUBDIRECTORY(mock)
 ADD_SUBDIRECTORY(src)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/bootstrap

@@ -45,7 +45,7 @@ Dependencies:
     protobuf        https://code.google.com/p/protobuf/
     kerberos        http://web.mit.edu/kerberos/
     libgsasl        http://www.gnu.org/software/gsasl/
-    libxml2         http://xmlsoft.org/
+    expat           http://expat.sourceforge.net/
     libuuid         http://sourceforge.net/projects/libuuid/
     boost 1.53+     http://www.boost.org/
                     [boost is not required if c++ compiler is g++ 4.6.0+ or clang++ with stdc++]

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt

@@ -106,16 +106,19 @@ INCLUDE_DIRECTORIES(${LIBXML2_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${KERBEROS_INCLUDE_DIRS})
 INCLUDE_DIRECTORIES(${GSASL_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/mock)
+INCLUDE_DIRECTORIES(${EXPAT_INCLUDE_DIR})
 
 TARGET_LINK_LIBRARIES(libhdfs3-static ${PROTOBUF_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${LIBXML2_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${KERBEROS_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${GSASL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-static ${EXPAT_LIBRARIES})
 
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${PROTOBUF_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${LIBXML2_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${KERBEROS_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${GSASL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-shared ${EXPAT_LIBRARIES})
 
 SET_TARGET_PROPERTIES(libhdfs3-static PROPERTIES OUTPUT_NAME "hdfs3")
 SET_TARGET_PROPERTIES(libhdfs3-shared PROPERTIES OUTPUT_NAME "hdfs3")

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.cc

@@ -131,7 +131,7 @@ static std::string ExtractPrincipalFromToken(const Token &token) {
 }
 }
 
-FileSystem::FileSystem(const Config &conf) : conf(conf) {
+FileSystem::FileSystem(const Config &conf) : conf_(conf) {
 }
 
 FileSystem::~FileSystem() {
@@ -140,7 +140,7 @@ FileSystem::~FileSystem() {
 
 Status FileSystem::connect() {
     try {
-        internal::SessionConfig sconf(*conf.impl);
+        internal::SessionConfig sconf(conf_);
         return connect(sconf.getDefaultUri().c_str(), NULL, NULL);
     } catch (...) {
         return CreateStatusFromException(current_exception());
@@ -183,14 +183,14 @@ Status FileSystem::connect(const std::string &uri, const std::string &username,
     CHECK_PARAMETER(!impl, EIO, "FileSystem: already connected.");
 
     try {
-        SessionConfig sconf(*conf.impl);
+        SessionConfig sconf(conf_);
         auth = RpcAuth::ParseMethod(sconf.getRpcAuthMethod());
 
         if (!token.empty() && auth != AuthMethod::SIMPLE) {
             Token t;
             t.fromString(token);
             principal = ExtractPrincipalFromToken(t);
-            impl = ConnectInternal(uri, principal, &t, conf);
+            impl = ConnectInternal(uri, principal, &t, conf_);
             impl->connect();
             return Status::OK();
         } else if (!username.empty()) {
@@ -202,7 +202,7 @@ Status FileSystem::connect(const std::string &uri, const std::string &username,
                 ExtractPrincipalFromTicketCache(sconf.getKerberosCachePath());
         }
 
-        impl = ConnectInternal(uri, principal, NULL, conf);
+        impl = ConnectInternal(uri, principal, NULL, conf_);
         impl->connect();
     } catch (...) {
         impl.reset();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.h

@@ -267,7 +267,7 @@ public:
 private:
     FileSystem(const FileSystem &other);
     FileSystem &operator=(const FileSystem &other);
-    Config conf;
+    Config conf_;
     hdfs::internal::shared_ptr<internal::FileSystemImpl> impl;
 
     friend class InputStream;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc

@@ -34,7 +34,6 @@
 #include <cstring>
 #include <deque>
 #include <inttypes.h>
-#include <libxml/uri.h>
 #include <strings.h>
 
 using std::string;
@@ -99,7 +98,7 @@ FileSystemImpl::FileSystemImpl(const FileSystemKey &key, const Config &c)
       key(key),
       leaseRenewer(this),
       nn(NULL),
-      sconf(*c.impl),
+      sconf(c),
       user(key.getUser()) {
     static atomic<uint32_t> count(0);
     std::stringstream ss;

+ 108 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc

@@ -0,0 +1,108 @@
+/**
+ * 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 "Atoi.h"
+#include "Status.h"
+
+#include <limits>
+#include <sstream>
+#include <stdint.h>
+#include <stdlib.h>
+#include <string>
+
+using hdfs::Status;
+using std::ostringstream;
+
+Status StrToInt32(const char *str, int32_t *ret) {
+    long retval;
+    char *end = NULL;
+    errno = 0;
+    retval = strtol(str, &end, 0);
+
+    if (EINVAL == errno || 0 != *end) {
+        ostringstream oss;
+        oss << "Invalid int32_t type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    if (ERANGE == errno || retval > std::numeric_limits<int32_t>::max() ||
+        retval < std::numeric_limits<int32_t>::min()) {
+        ostringstream oss;
+        oss << "Underflow/Overflow in int32_t type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    *ret = retval;
+    return Status::OK();
+}
+
+Status StrToInt64(const char *str, int64_t *ret) {
+    long long retval;
+    char *end = NULL;
+    errno = 0;
+    retval = strtoll(str, &end, 0);
+
+    if (EINVAL == errno || 0 != *end) {
+        ostringstream oss;
+        oss << "Invalid int64_t type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    if (ERANGE == errno || retval > std::numeric_limits<int64_t>::max() ||
+        retval < std::numeric_limits<int64_t>::min()) {
+        ostringstream oss;
+        oss << "Underflow/Overflow in int64_t type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    *ret = retval;
+    return Status::OK();
+}
+
+Status StrToBool(const char *str, bool *ret) {
+    bool retval = false;
+
+    if (!strcasecmp(str, "true") || !strcmp(str, "1")) {
+        retval = true;
+    } else if (!strcasecmp(str, "false") || !strcmp(str, "0")) {
+        retval = false;
+    } else {
+        ostringstream oss;
+        oss << "Invalid bool type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    *ret = retval;
+    return Status::OK();
+}
+
+Status StrToDouble(const char *str, double *ret) {
+    double retval;
+    char *end = NULL;
+    errno = 0;
+    retval = strtod(str, &end);
+
+    if (EINVAL == errno || 0 != *end) {
+        ostringstream oss;
+        oss << "Invalid double type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    if (ERANGE == errno || retval > std::numeric_limits<double>::max() ||
+        retval < std::numeric_limits<double>::min()) {
+        ostringstream oss;
+        oss << "Underflow/Overflow in double type: " << str;
+        return Status(EINVAL, oss.str());
+    }
+    *ret = retval;
+    return Status::OK();
+}

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h

@@ -0,0 +1,40 @@
+/**
+ * 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 _HDFS_LIBHDFS3_COMMON_ATOI_H_
+#define _HDFS_LIBHDFS3_COMMON_ATOI_H_
+
+#include "StatusInternal.h"
+
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+  Status StrToInt32(const char *str, int32_t *ret);
+
+  Status StrToInt64(const char *str, int64_t *ret);
+
+  Status StrToBool(const char *str, bool *ret);
+
+  Status StrToDouble(const char *str, double *ret);
+
+}
+}
+
+#endif

+ 106 - 122
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.cc

@@ -16,198 +16,182 @@
  * limitations under the License.
  */
 
+#include "Atoi.h"
 #include "Config.h"
-#include "ConfigImpl.h"
-#include "XmlConfigParser.h"
 #include "StatusInternal.h"
 
-using namespace hdfs::internal;
-
-namespace hdfs {
-
-Config Config::CreateFromXmlFile(const std::string &path) {
-    return Config(new ConfigImpl(XmlConfigParser(path.c_str()).getKeyValue()));
-}
-
-Config::Config() : impl(new ConfigImpl) {
-}
-
-Config::Config(const Config &other) {
-    impl = new ConfigImpl(*other.impl);
-}
+#include <inttypes.h>
+#include <map>
+#include <stdint.h>
+#include <stdlib.h>
+#include <string>
 
-Config::Config(ConfigImpl *impl) : impl(impl) {
-}
-
-Config &Config::operator=(const Config &other) {
-    if (this == &other) {
-        return *this;
-    }
+using namespace hdfs::internal;
+using std::string;
 
-    ConfigImpl *temp = impl;
-    impl = new ConfigImpl(*other.impl);
-    delete temp;
-    return *this;
-}
+typedef std::map<std::string, std::string> map_t;
 
-bool Config::operator==(const Config &other) const {
-    if (this == &other) {
-        return true;
-    }
 
-    return *impl == *other.impl;
-}
+namespace hdfs {
 
-Config::~Config() {
-    delete impl;
+Config::Config() {
 }
 
 Status Config::getString(const std::string &key, std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getString(key.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+    map_t::const_iterator i = map_.find(key);
+    if (i == map_.end()) {
+        return Status(ENOENT, "Configuration key " + key + " was not found.");
     }
-
+    *output = i->second;
     return Status::OK();
 }
 
 Status Config::getString(const std::string &key, const std::string &def,
                          std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getString(key.c_str(), def.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+    map_t::const_iterator i = map_.find(key);
+    if (i == map_.end()) {
+        *output = def;
     }
-
+    *output = i->second;
     return Status::OK();
 }
 
-Status Config::getInt64(const std::string &key, std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getInt64(key.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+Status Config::getInt64(const std::string &key, int64_t *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        return status;
+    }
+    status = StrToInt64(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
 Status Config::getInt64(const std::string &key, int64_t def,
-                        std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getInt64(key.c_str(), def);
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+                        int64_t *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        *output = def;
+        return Status::OK();
+    }
+    status = StrToInt64(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
-Status Config::getInt32(const std::string &key, std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getInt32(key.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+Status Config::getInt32(const std::string &key, int32_t *output) const {
+    std::string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        return status;
+    }
+    status = StrToInt32(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
 Status Config::getInt32(const std::string &key, int32_t def,
-                        std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getInt32(key.c_str(), def);
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+                        int32_t *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        *output = def;
+        return Status::OK();
+    }
+    status = StrToInt32(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
-Status Config::getDouble(const std::string &key, std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getDouble(key.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+Status Config::getDouble(const std::string &key, double *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        return status;
+    }
+    status = StrToDouble(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
 Status Config::getDouble(const std::string &key, double def,
-                         std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getDouble(key.c_str(), def);
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+                         double *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        *output = def;
+        return Status::OK();
+    }
+    status = StrToDouble(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
-Status Config::getBool(const std::string &key, std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getBool(key.c_str());
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+Status Config::getBool(const std::string &key, bool *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        return status;
+    }
+    status = StrToBool(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
 Status Config::getBool(const std::string &key, bool def,
-                       std::string *output) const {
-    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
-
-    try {
-        *output = impl->getBool(key.c_str(), def);
-    } catch (...) {
-        return CreateStatusFromException(current_exception());
+                       bool *output) const {
+    string out;
+    Status status = getString(key, &out);
+    if (status.isError()) {
+        *output = def;
+        return Status::OK();
+    }
+    status = StrToBool(out.c_str(), output);
+    if (status.isError()) {
+        return status;
     }
-
     return Status::OK();
 }
 
 void Config::set(const std::string &key, const std::string &value) {
-    impl->set(key.c_str(), value);
+    map_[key] = value;
 }
 
 void Config::set(const std::string &key, int32_t value) {
-    impl->set(key.c_str(), value);
+    char buf[32];
+    snprintf(buf, sizeof(buf), "%" PRId32, value);
+    set(key.c_str(), string(buf));
 }
 
 void Config::set(const std::string &key, int64_t value) {
-    impl->set(key.c_str(), value);
+    char buf[64];
+    snprintf(buf, sizeof(buf), "%" PRId64, value);
+    set(key.c_str(), string(buf));
 }
 
 void Config::set(const std::string &key, double value) {
-    impl->set(key.c_str(), value);
+    char buf[64];
+    snprintf(buf, sizeof(buf), "%f", value);
+    set(key.c_str(), string(buf));
 }
 
 void Config::set(const std::string &key, bool value) {
-    impl->set(key.c_str(), value);
+    set(key.c_str(), value ? string("true") : string("false"));
 }
 
-size_t Config::hash_value() const {
-    return impl->hash_value();
-}
 }

+ 10 - 54
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.h

@@ -27,49 +27,15 @@
 
 namespace hdfs {
 
-class FileSystem;
-class NamenodeInfo;
-
-namespace internal {
-class ConfigImpl;
-class FileSystemImpl;
-}
-
 /**
  * A configure file parser.
  */
 class Config {
 public:
-    /**
-     * Create an instance from a XML file
-     * @param path the path of the configure file.
-     */
-    static Config CreateFromXmlFile(const std::string &path);
-
     /**
      * Construct a empty Config instance.
      */
-    Config();
-
-    /**
-     * Copy constructor
-     */
-    Config(const Config &other);
-
-    /**
-     * Assignment operator.
-     */
-    Config &operator=(const Config &other);
-
-    /**
-     * Operator equal
-     */
-    bool operator==(const Config &other) const;
-
-    /**
-     * Destroy this instance
-     */
-    ~Config();
+    explicit Config();
 
     /**
      * Get a string with given configure key.
@@ -96,7 +62,7 @@ public:
      * @param output the pointer of the output parameter.
      * @return the result status of this operation
      */
-    Status getInt64(const std::string &key, std::string *output) const;
+    Status getInt64(const std::string &key, int64_t *output) const;
 
     /**
      * Get a 64 bit integer with given configure key.
@@ -107,7 +73,7 @@ public:
      * @return the result status of this operation
      */
     Status getInt64(const std::string &key, int64_t def,
-                    std::string *output) const;
+                    int64_t *output) const;
 
     /**
      * Get a 32 bit integer with given configure key.
@@ -115,7 +81,7 @@ public:
      * @param output the pointer of the output parameter.
      * @return the result status of this operation
      */
-    Status getInt32(const std::string &key, std::string *output) const;
+    Status getInt32(const std::string &key, int32_t *output) const;
 
     /**
      * Get a 32 bit integer with given configure key.
@@ -126,7 +92,7 @@ public:
      * @return the result status of this operation
      */
     Status getInt32(const std::string &key, int32_t def,
-                    std::string *output) const;
+                    int32_t *output) const;
 
     /**
      * Get a double with given configure key.
@@ -134,7 +100,7 @@ public:
      * @param output the pointer of the output parameter.
      * @return the result status of this operation
      */
-    Status getDouble(const std::string &key, std::string *output) const;
+    Status getDouble(const std::string &key, double *output) const;
 
     /**
      * Get a double with given configure key.
@@ -145,7 +111,7 @@ public:
      * @return the result status of this operation
      */
     Status getDouble(const std::string &key, double def,
-                     std::string *output) const;
+                     double *output) const;
 
     /**
      * Get a boolean with given configure key.
@@ -153,7 +119,7 @@ public:
      * @param output the pointer of the output parameter.
      * @return the result status of this operation
      */
-    Status getBool(const std::string &key, std::string *output) const;
+    Status getBool(const std::string &key, bool *output) const;
 
     /**
      * Get a boolean with given configure key.
@@ -163,7 +129,7 @@ public:
      * @param output the pointer of the output parameter.
      * @return the result status of this operation
      */
-    Status getBool(const std::string &key, bool def, std::string *output) const;
+    Status getBool(const std::string &key, bool def, bool *output) const;
 
     /**
      * Set a configure item
@@ -200,18 +166,8 @@ public:
      */
     void set(const std::string &key, bool value);
 
-    /**
-     * Get the hash value of this object
-     * @return The hash value
-     */
-    size_t hash_value() const;
-
 private:
-    Config(hdfs::internal::ConfigImpl *impl);
-    hdfs::internal::ConfigImpl *impl;
-    friend class hdfs::FileSystem;
-    friend class hdfs::internal::FileSystemImpl;
-    friend class hdfs::NamenodeInfo;
+    std::map<std::string, std::string> map_;
 };
 }
 

+ 0 - 291
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.cc

@@ -1,291 +0,0 @@
-/**
- * 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 "Exception.h"
-#include "ExceptionInternal.h"
-#include "Hash.h"
-#include "ConfigImpl.h"
-
-#include <cassert>
-#include <errno.h>
-#include <fstream>
-#include <limits>
-#include <string.h>
-#include <unistd.h>
-#include <vector>
-
-using std::map;
-using std::string;
-using std::vector;
-
-namespace hdfs {
-namespace internal {
-
-typedef map<string, string>::const_iterator Iterator;
-typedef map<string, string> Map;
-
-static int32_t StrToInt32(const char *str) {
-    long retval;
-    char *end = NULL;
-    errno = 0;
-    retval = strtol(str, &end, 0);
-
-    if (EINVAL == errno || 0 != *end) {
-        THROW(HdfsBadNumFoumat, "Invalid int32_t type: %s", str);
-    }
-
-    if (ERANGE == errno || retval > std::numeric_limits<int32_t>::max() ||
-        retval < std::numeric_limits<int32_t>::min()) {
-        THROW(HdfsBadNumFoumat, "Underflow/Overflow int32_t type: %s", str);
-    }
-
-    return retval;
-}
-
-static int64_t StrToInt64(const char *str) {
-    long long retval;
-    char *end = NULL;
-    errno = 0;
-    retval = strtoll(str, &end, 0);
-
-    if (EINVAL == errno || 0 != *end) {
-        THROW(HdfsBadNumFoumat, "Invalid int64_t type: %s", str);
-    }
-
-    if (ERANGE == errno || retval > std::numeric_limits<int64_t>::max() ||
-        retval < std::numeric_limits<int64_t>::min()) {
-        THROW(HdfsBadNumFoumat, "Underflow/Overflow int64_t type: %s", str);
-    }
-
-    return retval;
-}
-
-static bool StrToBool(const char *str) {
-    bool retval = false;
-
-    if (!strcasecmp(str, "true") || !strcmp(str, "1")) {
-        retval = true;
-    } else if (!strcasecmp(str, "false") || !strcmp(str, "0")) {
-        retval = false;
-    } else {
-        THROW(HdfsBadBoolFoumat, "Invalid bool type: %s", str);
-    }
-
-    return retval;
-}
-
-static double StrToDouble(const char *str) {
-    double retval;
-    char *end = NULL;
-    errno = 0;
-    retval = strtod(str, &end);
-
-    if (EINVAL == errno || 0 != *end) {
-        THROW(HdfsBadNumFoumat, "Invalid double type: %s", str);
-    }
-
-    if (ERANGE == errno || retval > std::numeric_limits<double>::max() ||
-        retval < std::numeric_limits<double>::min()) {
-        THROW(HdfsBadNumFoumat, "Underflow/Overflow int64_t type: %s", str);
-    }
-
-    return retval;
-}
-
-ConfigImpl::ConfigImpl(const Map &kv) : kv(kv) {
-}
-
-const char *ConfigImpl::getString(const std::string &key) const {
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found", key.c_str());
-    }
-
-    return it->second.c_str();
-}
-
-const char *ConfigImpl::getString(const std::string &key,
-                                  const std::string &def) const {
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        return def.c_str();
-    } else {
-        return it->second.c_str();
-    }
-}
-
-int64_t ConfigImpl::getInt64(const std::string &key) const {
-    int64_t retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found", key.c_str());
-    }
-
-    try {
-        retval = StrToInt64(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-int64_t ConfigImpl::getInt64(const std::string &key, int64_t def) const {
-    int64_t retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToInt64(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-int32_t ConfigImpl::getInt32(const std::string &key) const {
-    int32_t retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found", key.c_str());
-    }
-
-    try {
-        retval = StrToInt32(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-int32_t ConfigImpl::getInt32(const std::string &key, int32_t def) const {
-    int32_t retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToInt32(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-double ConfigImpl::getDouble(const std::string &key) const {
-    double retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found", key.c_str());
-    }
-
-    try {
-        retval = StrToDouble(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-double ConfigImpl::getDouble(const std::string &key, double def) const {
-    double retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToDouble(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-bool ConfigImpl::getBool(const std::string &key) const {
-    bool retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found", key.c_str());
-    }
-
-    try {
-        retval = StrToBool(it->second.c_str());
-    } catch (const HdfsBadBoolFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-bool ConfigImpl::getBool(const std::string &key, bool def) const {
-    bool retval;
-    Iterator it = kv.find(key.c_str());
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToBool(it->second.c_str());
-    } catch (const HdfsBadNumFoumat &e) {
-        NESTED_THROW(HdfsConfigNotFound, "ConfigImpl key: %s not found",
-                     key.c_str());
-    }
-
-    return retval;
-}
-
-size_t ConfigImpl::hash_value() const {
-    vector<size_t> values;
-    map<string, string>::const_iterator s, e;
-    e = kv.end();
-
-    for (s = kv.begin(); s != e; ++s) {
-        values.push_back(StringHasher(s->first));
-        values.push_back(StringHasher(s->second));
-    }
-
-    return CombineHasher(&values[0], values.size());
-}
-}
-}

+ 0 - 165
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.h

@@ -1,165 +0,0 @@
-/**
- * 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 _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_
-#define _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_
-
-#include <stdint.h>
-#include <string>
-#include <sstream>
-#include <map>
-
-namespace hdfs {
-namespace internal {
-
-/**
- * A configure file parser.
- */
-class ConfigImpl {
-public:
-    ConfigImpl() {
-    }
-
-    /**
-     * Construct a empty Config instance.
-     */
-    ConfigImpl(const std::map<std::string, std::string> &kv);
-
-    /**
-     * Operator equal
-     */
-    bool operator==(const ConfigImpl &other) const {
-        if (this == &other) {
-            return true;
-        }
-
-        return this->kv == other.kv;
-    }
-
-    /**
-     * Get a string with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
-     */
-    const char *getString(const std::string &key) const;
-
-    /**
-     * Get a string with given configure key.
-     * Return the default value def if key is not found.
-     * @param key The key of the configure item.
-     * @param def The defalut value.
-     * @return The value of configure item.
-     */
-    const char *getString(const std::string &key, const std::string &def) const;
-
-    /**
-     * Get a 64 bit integer with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
-     */
-    int64_t getInt64(const std::string &key) const;
-
-    /**
-     * Get a 64 bit integer with given configure key.
-     * Return the default value def if key is not found.
-     * @param key The key of the configure item.
-     * @param def The defalut value.
-     * @return The value of configure item.
-     */
-    int64_t getInt64(const std::string &key, int64_t def) const;
-
-    /**
-     * Get a 32 bit integer with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
-     */
-    int32_t getInt32(const std::string &key) const;
-
-    /**
-     * Get a 32 bit integer with given configure key.
-     * Return the default value def if key is not found.
-     * @param key The key of the configure item.
-     * @param def The defalut value.
-     * @return The value of configure item.
-     */
-    int32_t getInt32(const std::string &key, int32_t def) const;
-
-    /**
-     * Get a double with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
-     */
-    double getDouble(const std::string &key) const;
-
-    /**
-     * Get a double with given configure key.
-     * Return the default value def if key is not found.
-     * @param key The key of the configure item.
-     * @param def The defalut value.
-     * @return The value of configure item.
-     */
-    double getDouble(const std::string &key, double def) const;
-
-    /**
-     * Get a boolean with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
-     */
-    bool getBool(const std::string &key) const;
-
-    /**
-     * Get a boolean with given configure key.
-     * Return the default value def if key is not found.
-     * @param key The key of the configure item.
-     * @param def The default value.
-     * @return The value of configure item.
-     */
-    bool getBool(const std::string &key, bool def) const;
-
-    /**
-     * Set a configure item
-     * @param key The key will set.
-     * @param value The value will be set to.
-     */
-    template <typename T>
-    void set(const std::string &key, T const &value) {
-        std::stringstream ss;
-        ss << value;
-        kv[key] = ss.str();
-    }
-
-    /**
-     * Get the hash value of this object
-     *
-     * @return The hash value
-     */
-    size_t hash_value() const;
-
-private:
-    std::string path;
-    std::map<std::string, std::string> kv;
-};
-}
-}
-
-#endif /* _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_ */

+ 27 - 11
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.cc

@@ -18,14 +18,17 @@
 
 #include "SessionConfig.h"
 
-#include <sstream>
-
+#include "Config.h"
 #include "Exception.h"
 #include "ExceptionInternal.h"
 #include "Function.h"
 
+#include <sstream>
+
 #define ARRAYSIZE(A) (sizeof(A) / sizeof(A[0]))
 
+using hdfs::Config;
+
 namespace hdfs {
 namespace internal {
 
@@ -48,7 +51,7 @@ static void CheckMultipleOf(const char *key, const T &value, int unit) {
     }
 }
 
-SessionConfig::SessionConfig(const ConfigImpl &conf) {
+SessionConfig::SessionConfig(const Config &conf) {
     ConfigDefault<bool> boolValues[] = {
         {&rpcTcpNoDelay, "rpc.client.connect.tcpnodelay", true},
         {&readFromLocal, "dfs.client.read.shortcircuit", true},
@@ -104,8 +107,11 @@ SessionConfig::SessionConfig(const ConfigImpl &conf) {
         {&logSeverity, "dfs.client.log.severity", "INFO"}};
 
     for (size_t i = 0; i < ARRAYSIZE(boolValues); ++i) {
-        *boolValues[i].variable =
-            conf.getBool(boolValues[i].key, boolValues[i].value);
+        Status status = conf.getBool(boolValues[i].key,
+                              boolValues[i].value, boolValues[i].variable);
+        if (status.isError()) {
+            THROW(HdfsConfigInvalid, "%s", status.getErrorMsg());
+        }
 
         if (boolValues[i].check) {
             boolValues[i].check(boolValues[i].key, *boolValues[i].variable);
@@ -113,8 +119,11 @@ SessionConfig::SessionConfig(const ConfigImpl &conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(i32Values); ++i) {
-        *i32Values[i].variable =
-            conf.getInt32(i32Values[i].key, i32Values[i].value);
+        Status status = conf.getInt32(i32Values[i].key,
+                              i32Values[i].value, i32Values[i].variable);
+        if (status.isError()) {
+            THROW(HdfsConfigInvalid, "%s", status.getErrorMsg());
+        }
 
         if (i32Values[i].check) {
             i32Values[i].check(i32Values[i].key, *i32Values[i].variable);
@@ -122,8 +131,11 @@ SessionConfig::SessionConfig(const ConfigImpl &conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(i64Values); ++i) {
-        *i64Values[i].variable =
-            conf.getInt64(i64Values[i].key, i64Values[i].value);
+        Status status = conf.getInt64(i64Values[i].key,
+                              i64Values[i].value, i64Values[i].variable);
+        if (status.isError()) {
+            THROW(HdfsConfigInvalid, "%s", status.getErrorMsg());
+        }
 
         if (i64Values[i].check) {
             i64Values[i].check(i64Values[i].key, *i64Values[i].variable);
@@ -131,8 +143,12 @@ SessionConfig::SessionConfig(const ConfigImpl &conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(strValues); ++i) {
-        *strValues[i].variable =
-            conf.getString(strValues[i].key, strValues[i].value.c_str());
+        Status status = conf.getString(strValues[i].key,
+                              strValues[i].value.c_str(),
+                              strValues[i].variable);
+        if (status.isError()) {
+            THROW(HdfsConfigInvalid, "%s", status.getErrorMsg());
+        }
 
         if (strValues[i].check) {
             strValues[i].check(strValues[i].key, *strValues[i].variable);

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.h

@@ -19,7 +19,6 @@
 #ifndef _HDFS_LIBHDFS3_COMMON_SESSIONCONFIG_H_
 #define _HDFS_LIBHDFS3_COMMON_SESSIONCONFIG_H_
 
-#include "ConfigImpl.h"
 #include "Exception.h"
 #include "ExceptionInternal.h"
 #include "Function.h"
@@ -27,7 +26,10 @@
 
 #include <cassert>
 #include <stdint.h>
-#include <vector>
+
+namespace hdfs {
+class Config;
+}
 
 namespace hdfs {
 namespace internal {
@@ -43,7 +45,7 @@ struct ConfigDefault {
 
 class SessionConfig {
 public:
-    SessionConfig(const ConfigImpl &conf);
+    explicit SessionConfig(const hdfs::Config &conf);
 
     /*
      * rpc configure

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h

@@ -57,4 +57,12 @@ private:
 };
 }
 
+#define RETURN_NOT_OK(expr) \
+    do { \
+        Status s = expr; \
+        if (s.isError()) { \
+            return s; \
+        } \
+    } while (0);
+
 #endif /* _HDFS_LIBHDFS3_COMMON_STATUS_H_ */

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StringUtil.h

@@ -24,6 +24,14 @@
 #include <vector>
 #include <cctype>
 
+const char *const HDFS_XML_NAMES[] = {
+    "core-default.xml",
+    "core-site.xml",
+    "hdfs-default.xml",
+    "hdfs-site.xml",
+    NULL
+};
+
 namespace hdfs {
 namespace internal {
 

+ 269 - 91
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.cc

@@ -16,139 +16,317 @@
  * limitations under the License.
  */
 
+#include "Atoi.h"
+#include "Config.h"
 #include "Exception.h"
 #include "ExceptionInternal.h"
+#include "Logger.h"
+#include "SharedPtr.h"
+#include "StatusInternal.h"
+#include "StringUtil.h"
 #include "XmlConfigParser.h"
 
 #include <cassert>
 #include <errno.h>
+#include <expat.h>
 #include <fstream>
-#include <libxml/parser.h>
-#include <libxml/tree.h>
 #include <limits>
+#include <map>
 #include <string.h>
 #include <unistd.h>
 #include <vector>
 
 using namespace hdfs::internal;
-
-using std::map;
 using std::string;
 using std::vector;
 
 namespace hdfs {
 namespace internal {
 
-typedef map<string, string>::const_iterator Iterator;
-typedef map<string, string> Map;
+enum XmlParseState {
+    HCXML_PARSE_INIT = 0,
+    HCXML_PARSE_IN_CONFIG,
+    HCXML_PARSE_IN_PROPERTY,
+    HCXML_PARSE_IN_NAME,
+    HCXML_PARSE_IN_VALUE,
+    HCXML_PARSE_IN_FINAL,
+};
+
+// TODO: deprecation table
+
+struct Value {
+    Value() : final_(false) {
+    }
+
+    Value(const std::string &text, bool f)
+        : text_(text), final_(f) { }
+    std::string text_;
+    bool final_;
+};
+
+typedef std::map<std::string, Value> accum_map_t;
 
-static void readXmlConfigParserItem(xmlNodePtr root, Map &kv,
-                                    const char *path) {
-    std::string key, value;
-    xmlNodePtr curNode;
-    bool hasname = false, hasvalue = false;
+class XmlData {
+public:
+    XmlData(const std::string &path, XML_Parser parser) 
+            : path_(path),
+              parser_(parser),
+              state_(HCXML_PARSE_INIT),
+              ignoredParents_(0),
+              final_(false) {
+        XML_SetUserData(parser_, this);
+        XML_SetElementHandler(parser_, XmlData::startElement, XmlData::endElement);
+        XML_SetCharacterDataHandler(parser_, handleData);
+    }
+
+    ~XmlData() {
+        XML_ParserFree(parser_);
+    }
 
-    for (curNode = root; NULL != curNode; curNode = curNode->next) {
-        if (curNode->type != XML_ELEMENT_NODE) {
-            continue;
+    void populateConf(hdfs::Config *conf) {
+        for (accum_map_t::iterator i = accumMap_.begin();
+                 i != accumMap_.end(); ++i) {
+            conf->set(i->first, i->second.text_);
         }
+    }
+
+private:
+    XmlData(const XmlData&);
+    XmlData &operator=(const XmlData &);
 
-        if (!hasname && !strcmp((const char *)curNode->name, "name")) {
-            if (NULL != curNode->children &&
-                XML_TEXT_NODE == curNode->children->type) {
-                key = (const char *)curNode->children->content;
-                hasname = true;
+    static void startElement(void *data, const char *element,
+                const char **attribute) {
+        XmlData *ctx = reinterpret_cast<XmlData*>(data);
+        if (ctx->ignoredParents_ > 0) {
+            ctx->ignoredParents_++;
+            return;
+        }
+        switch (ctx->state_) {
+        case HCXML_PARSE_INIT:
+            if (!strcmp(element, "configuration")) {
+                ctx->state_ = HCXML_PARSE_IN_CONFIG;
+                return;
             }
-        } else if (!hasvalue && !strcmp((const char *)curNode->name, "value")) {
-            if (NULL != curNode->children &&
-                XML_TEXT_NODE == curNode->children->type) {
-                value = (const char *)curNode->children->content;
-                hasvalue = true;
+            break;
+        case HCXML_PARSE_IN_CONFIG:
+            if (!strcmp(element, "property")) {
+                ctx->state_ = HCXML_PARSE_IN_PROPERTY;
+                return;
             }
-        } else {
-            continue;
+            break;
+        case HCXML_PARSE_IN_PROPERTY:
+            if (!strcmp(element, "name")) {
+                ctx->state_ = HCXML_PARSE_IN_NAME;
+                return;
+            } else if (!strcmp(element, "value")) {
+                ctx->state_ = HCXML_PARSE_IN_VALUE;
+                return;
+            } else if (!strcmp(element, "final")) {
+                ctx->state_ = HCXML_PARSE_IN_FINAL;
+                return;
+            }
+            break;
+        default:
+            break;
         }
+        LOG(LOG_ERROR, "XmlData(%s): ignoring "
+                "element '%s'\n", ctx->path_.c_str(), element);
+        ctx->ignoredParents_++;
     }
 
-    if (hasname && hasvalue) {
-        kv[key] = value;
-        return;
-    } else if (hasname) {
-        kv[key] = "";
-        return;
-    }
+    static void endElement(void *data, const char *el) {
+        XmlData *ctx = reinterpret_cast<XmlData*>(data);
 
-    THROW(HdfsBadConfigFoumat,
-          "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"", path);
-}
+        if (ctx->ignoredParents_ > 0) {
+            ctx->ignoredParents_--;
+            return;
+        }
+        switch (ctx->state_) {
+        case HCXML_PARSE_IN_CONFIG:
+            ctx->state_ = HCXML_PARSE_INIT;
+            break;
+        case HCXML_PARSE_IN_PROPERTY:
+            ctx->state_ = HCXML_PARSE_IN_CONFIG;
+            if (ctx->name_.empty()) {
+                LOG(LOG_ERROR, "hconf_builder_load_xml(%s): property "
+                    "tag is missing <name> on line %lld\n",
+                    ctx->path_.c_str(),
+                    (long long)XML_GetCurrentLineNumber(ctx->parser_));
+            } else if (ctx->value_.empty()) {
+                LOG(LOG_ERROR, "hconf_builder_load_xml(%s): property "
+                    "tag is missing <value> on line %lld\n",
+                    ctx->path_.c_str(),
+                    (long long)XML_GetCurrentLineNumber(ctx->parser_));
+            } else {
+                accum_map_t::iterator i = ctx->accumMap_.find(ctx->name_);
+                if ((i != ctx->accumMap_.end()) && (i->second.final_)) {
+                    LOG(LOG_ERROR, "XmlData(%s): ignoring attempt to "
+                        "re-set final key '%s'\n", ctx->path_.c_str(),
+                        i->second.text_.c_str());
+                    break;
+                }
+                ctx->accumMap_[ctx->name_] = Value(ctx->name_, ctx->final_);
+            }
+            ctx->name_.clear();
+            ctx->value_.clear();
+            ctx->final_ = false;
+            break;
+        case HCXML_PARSE_IN_NAME:
+            ctx->state_ = HCXML_PARSE_IN_PROPERTY;
+            break;
+        case HCXML_PARSE_IN_VALUE:
+            ctx->state_ = HCXML_PARSE_IN_PROPERTY;
+            break;
+        case HCXML_PARSE_IN_FINAL:
+            ctx->state_ = HCXML_PARSE_IN_PROPERTY;
+            break;
+        default:
+            break;
+        }
+    }
 
-static void readXmlConfigParserItems(xmlDocPtr doc, Map &kv, const char *path) {
-    xmlNodePtr root, curNode;
-    root = xmlDocGetRootElement(doc);
+    static void handleData(void *data, const char *content, int length) {
+        XmlData *ctx = reinterpret_cast<XmlData*>(data);
 
-    if (NULL == root ||
-        strcmp((const char *)root->name, "XmlConfigParseruration")) {
-        THROW(HdfsBadConfigFoumat,
-              "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"",
-              path);
+        switch (ctx->state_) {
+        case HCXML_PARSE_IN_NAME: {
+            if (ctx->name_.empty()) {
+                ctx->name_ = string(content, length);
+            } else {
+                    LOG(LOG_ERROR, "XmlData(%s): duplicate "
+                        "<name> tag on line %lld\n", ctx->path_.c_str(),
+                        (long long)XML_GetCurrentLineNumber(ctx->parser_));
+                }
+                break;
+        }
+        case HCXML_PARSE_IN_VALUE: {
+            if (ctx->value_.empty()) {
+                ctx->value_ = string(content, length);
+            } else {
+                LOG(LOG_ERROR, "XmlData(%s): duplicate "
+                    "<value> tag on line %lld\n", ctx->path_.c_str(),
+                    (long long)XML_GetCurrentLineNumber(ctx->parser_));
+            }
+            break;
+        }
+        case HCXML_PARSE_IN_FINAL: {
+            string boolStr(content, length);
+            bool val;
+            Status status = StrToBool(boolStr.c_str(), &val);
+            if (status.isError()) {
+                LOG(LOG_ERROR, "XmlData(%s): error parsing "
+                    "<final> tag on line %lld\n", ctx->path_.c_str(),
+                    (long long)XML_GetCurrentLineNumber(ctx->parser_));
+            } else {
+                ctx->final_ = val;
+            }
+            break;
+        }
+        default:
+            break;
+        }
     }
 
-    /*
-     * for each property
-     */
-    for (curNode = root->children; NULL != curNode; curNode = curNode->next) {
-        if (curNode->type != XML_ELEMENT_NODE) {
-            continue;
-        }
+    /** Path of the current XML file we're parsing. */
+    std::string path_;
 
-        if (strcmp((const char *)curNode->name, "property")) {
-            THROW(
-                HdfsBadConfigFoumat,
-                "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"",
-                path);
-        }
+    /** The XML parser we're using. */
+    XML_Parser parser_;
 
-        readXmlConfigParserItem(curNode->children, kv, path);
-    }
-}
+    /** XML parse state. */
+    XmlParseState state_;
+
+    /** The number of parent elements we are ignoring. */
+    int ignoredParents_;
+
+    /** Nonzero if the current property is final. */
+    bool final_;
+
+    /** Malloced key, if we saw one. */
+    string name_;
+
+    /** Malloced value, if we saw one. */
+    string value_;
+
+    /** The keys and values we've seen. */
+    accum_map_t accumMap_;
+};
+
+static const char *const HDFS_XML_NAMES[] = {
+    "core-default.xml",
+    "core-site.xml",
+    "hdfs-default.xml",
+    "hdfs-site.xml",
+    NULL
+};
 
-XmlConfigParser::XmlConfigParser(const char *p) : path(p) {
-    update(p);
+XmlConfigParser::XmlConfigParser(hdfs::Config *conf)
+    : conf_(conf) {
 }
 
-void XmlConfigParser::update(const char *p) {
-    char msg[64];
-    xmlDocPtr doc; /* the resulting document tree */
-    LIBXML_TEST_VERSION
-    kv.clear();
-    path = p;
-
-    if (access(path.c_str(), R_OK)) {
-        strerror_r(errno, msg, sizeof(msg));
-        THROW(HdfsBadConfigFoumat,
-              "Cannot read XmlConfigParserure file: \"%s\", %s", path.c_str(),
-              msg);
+Status XmlConfigParser::ParseXmls(const std::string &pathList) {
+    string errors, prefix;
+    vector<string> paths = StringSplit(pathList, ":");
+    for (vector<string>::const_iterator p = paths.begin();
+             p != paths.end(); ++p) {
+        for (size_t x = 0; HDFS_XML_NAMES[x]; ++x) {
+            Status status = ParseXml(*p + "/" + HDFS_XML_NAMES[x]);
+            if (status.getCode() != ENOENT) {
+                errors = errors + prefix + status.getErrorMsg();
+                prefix = ", ";
+            }
+        }
+    }
+    if (errors.empty()) {
+        return Status::OK();
     }
+    return Status(EINVAL, errors);
+}
 
-    /* parse the file */
-    doc = xmlReadFile(path.c_str(), NULL, 0);
-
-    try {
-        /* check if parsing succeeded */
-        if (doc == NULL) {
-            THROW(
-                HdfsBadConfigFoumat,
-                "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"",
-                path.c_str());
-        } else {
-            readXmlConfigParserItems(doc, kv, path.c_str());
-            /* free up the resulting document */
-            xmlFreeDoc(doc);
+Status XmlConfigParser::ParseXml(const std::string &path) {
+    ::FILE *fp = ::fopen(path.c_str(), "r");
+    if (!fp) {
+        int err = errno;
+        if (err == ENOENT) {
+            return Status(ENOENT, "No such file as " + path);
         }
-    } catch (...) {
-        xmlFreeDoc(doc);
-        throw;
+        return Status(err);
+    }
+    shared_ptr<FILE> fpPtr(fp, ::fclose);
+    XML_Parser parser = XML_ParserCreate("UTF-8");
+    if (!parser) {
+        return Status(ENOMEM, "Failed to create libexpat XML parser for " +
+                      path);
     }
+    XmlData xmlData(path, parser);
+
+    vector<uint8_t> buf(16384, 0x0);
+    int res = 0;
+    do {
+        res = fread(&buf[0], 1, buf.size(), fp);
+        if (res <= 0) {
+            if (feof(fp)) {
+                res = 0;
+            } else {
+                int e = errno;
+                return Status(e, "Failed to read from configuration XML "
+                              "file " + path);
+            }
+        }
+        int pstatus = XML_Parse(parser,
+                reinterpret_cast<const char *>(&buf[0]), res,
+                res ? XML_FALSE : XML_TRUE);
+        if (pstatus != XML_STATUS_OK) {
+            enum XML_Error error = XML_GetErrorCode(parser);
+            return Status(EINVAL, "hconf_builder_load_xml(" + path + "): " +
+                            "parse error: " + XML_ErrorString(error));
+        }
+    } while (res);
+    xmlData.populateConf(conf_);
+    return Status::OK();
 }
+
 }
 }
+
+// vim: ts=4:sw=4:tw=79:et

+ 23 - 25
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.h

@@ -19,10 +19,13 @@
 #ifndef _HDFS_LIBHDFS3_COMMON_XMLCONFIGPARSER_H_
 #define _HDFS_LIBHDFS3_COMMON_XMLCONFIGPARSER_H_
 
-#include <stdint.h>
+#include "StatusInternal.h"
+
 #include <string>
-#include <sstream>
-#include <map>
+
+namespace hdfs {
+class Config;
+}
 
 namespace hdfs {
 namespace internal {
@@ -32,37 +35,32 @@ namespace internal {
  */
 class XmlConfigParser {
 public:
-    /**
-     * Construct a empty Config instance.
-     */
-    XmlConfigParser() {
-    }
+    XmlConfigParser(hdfs::Config *conf);
 
     /**
-     * Construct a Config with given configure file.
-     * @param path The path of configure file.
-     * @throw HdfsBadConfigFoumat
+     * Parse a colon-separated list of paths for HDFS configuration files.
+     * The format is the same as that of CLASSPATH.
+     *
+     * Even when an error is returned, we may still load some entries into the
+     * Config file.
      */
-    XmlConfigParser(const char *path);
+    Status ParseXmls(const std::string &pathList);
 
     /**
-     * Parse the configure file.
-     * @throw HdfsBadConfigFoumat
+     * Parse a single XML file.
+     *
+     * Even when an error is returned, we may still load some entries into the
+     * Config file.
      */
-    void update(const char *path);
-
-    /**
-     * Get Key Values
-     * @return Return the Key Value pairs.
-     */
-    std::map<std::string, std::string> getKeyValue() {
-        return kv;
-    }
+    Status ParseXml(const std::string &path);
 
 private:
-    std::string path;
-    std::map<std::string, std::string> kv;
+    XmlConfigParser(const XmlConfigParser &);
+    XmlConfigParser &operator=(const XmlConfigParser &);
+
+    hdfs::Config *conf_;
 };
+
 }
 }
 

+ 14 - 10
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc

@@ -17,7 +17,6 @@
  */
 
 #include "Config.h"
-#include "ConfigImpl.h"
 #include "NamenodeInfo.h"
 #include "StatusInternal.h"
 #include "StringUtil.h"
@@ -26,6 +25,7 @@
 #include <vector>
 
 using namespace hdfs::internal;
+using std::string;
 
 namespace hdfs {
 
@@ -38,28 +38,32 @@ const char *const DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
 const char *const DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
 
 Status NamenodeInfo::GetHANamenodeInfo(const std::string &service,
-                                       const Config &c,
+                                       const Config &conf,
                                        std::vector<NamenodeInfo> *output) {
-    ConfigImpl &conf = *c.impl;
     CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
 
     try {
+        std::string strNameNodes;
         std::vector<NamenodeInfo> &retval = *output;
-        std::string strNameNodes = StringTrim(
-            conf.getString(std::string(DFS_NAMENODE_HA) + "." + service));
+        RETURN_NOT_OK(conf.getString(
+                std::string(DFS_NAMENODE_HA) + "." + service, &strNameNodes));
         std::vector<std::string> nns = StringSplit(strNameNodes, ",");
         retval.resize(nns.size());
 
         for (size_t i = 0; i < nns.size(); ++i) {
-            std::string dfsRpcAddress =
+            std::string dfsRpcAddressKey =
                 StringTrim(std::string(DFS_NAMENODE_RPC_ADDRESS_KEY) + "." +
                            service + "." + StringTrim(nns[i]));
-            std::string dfsHttpAddress =
+            string rpcAddress;
+            RETURN_NOT_OK(conf.getString(dfsRpcAddressKey, "", &rpcAddress));
+            retval[i].setRpcAddr(StringTrim(rpcAddress));
+
+            std::string dfsHttpAddressKey =
                 StringTrim(std::string(DFS_NAMENODE_HTTP_ADDRESS_KEY) + "." +
                            service + "." + StringTrim(nns[i]));
-            retval[i].setRpcAddr(StringTrim(conf.getString(dfsRpcAddress, "")));
-            retval[i].setHttpAddr(
-                StringTrim(conf.getString(dfsHttpAddress, "")));
+            string httpAddress;
+            RETURN_NOT_OK(conf.getString(dfsHttpAddressKey, "", &httpAddress));
+            retval[i].setHttpAddr(StringTrim(httpAddress));
         }
 
     } catch (...) {