Quellcode durchsuchen

HDFS-7014. Implement input streams and file system functionality (zhwangzw via cmccabe)

Colin Patrick Mccabe vor 10 Jahren
Ursprung
Commit
e559ce0483
42 geänderte Dateien mit 5301 neuen und 903 gelöschten Zeilen
  1. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
  3. 87 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockLocation.h
  4. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h
  5. 87 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.cc
  6. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h
  7. 518 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.cc
  8. 277 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystem.h
  9. 760 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
  10. 478 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.h
  11. 89 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemKey.cc
  12. 97 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemKey.h
  13. 77 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemStats.h
  14. 123 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc
  15. 35 12
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h
  16. 919 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.cc
  17. 42 38
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h
  18. 15 13
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h
  19. 14 14
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h
  20. 5 8
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h
  21. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h
  22. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/TokenInternal.h
  23. 213 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.cc
  24. 218 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.h
  25. 291 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.cc
  26. 28 45
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.h
  27. 68 95
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.cc
  28. 9 11
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.h
  29. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.cc
  30. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
  31. 137 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StatusInternal.h
  32. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h
  33. 0 395
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfig.cc
  34. 154 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.cc
  35. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.h
  36. 6 13
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc
  37. 3 34
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h
  38. 123 87
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h
  39. 77 69
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h
  40. 31 19
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc
  41. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h
  42. 36 32
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h

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

@@ -48,7 +48,18 @@ SET(libhdfs3_PROTO_FILES ${libhdfs3_PROTO_FILES} PARENT_SCOPE)
 INCLUDE(GenerateProtobufs.cmake)
 INCLUDE_DIRECTORIES("${CMAKE_BINARY_DIR}")
 
-SET(HEADER client/hdfs.h)
+SET(HEADER 
+    client/BlockLocation.h
+    client/DirectoryIterator.h
+    client/FileStatus.h
+    client/FileSystem.h
+    client/FileSystemStats.h
+    client/InputStream.h
+    client/Permission.h
+    common/Config.h
+    common/SharedPtr.h
+    common/Status.h
+)
 
 ADD_LIBRARY(libhdfs3-static STATIC ${LIBHDFS3_SOURCES} ${LIBHDFS3_PROTO_SOURCES} ${LIBHDFS3_PROTO_HEADERS})
 ADD_LIBRARY(libhdfs3-shared SHARED ${LIBHDFS3_SOURCES} ${LIBHDFS3_PROTO_SOURCES} ${LIBHDFS3_PROTO_HEADERS})

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake

@@ -56,6 +56,8 @@ COPY_IF_CHANGED("${CMAKE_BINARY_DIR}/hdfs_pb"
     ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
     ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
     ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
 )
 
 AUTO_SOURCES(PB_SOURCES "*.proto" "RECURSE" "${CMAKE_BINARY_DIR}")

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockLocation.h

@@ -0,0 +1,87 @@
+/**
+ * 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_CLIENT_BLOCKLOCATION_H_
+#define _HDFS_LIBHDFS3_CLIENT_BLOCKLOCATION_H_
+
+#include <string>
+#include <vector>
+
+namespace hdfs {
+
+class BlockLocation {
+public:
+    bool isCorrupt() const {
+        return corrupt;
+    }
+
+    void setCorrupt(bool corrupt) {
+        this->corrupt = corrupt;
+    }
+
+    const std::vector<std::string> &getHosts() const {
+        return hosts;
+    }
+
+    void setHosts(const std::vector<std::string> &hosts) {
+        this->hosts = hosts;
+    }
+
+    int64_t getLength() const {
+        return length;
+    }
+
+    void setLength(int64_t length) {
+        this->length = length;
+    }
+
+    const std::vector<std::string> &getNames() const {
+        return names;
+    }
+
+    void setNames(const std::vector<std::string> &names) {
+        this->names = names;
+    }
+
+    int64_t getOffset() const {
+        return offset;
+    }
+
+    void setOffset(int64_t offset) {
+        this->offset = offset;
+    }
+
+    const std::vector<std::string> &getTopologyPaths() const {
+        return topologyPaths;
+    }
+
+    void setTopologyPaths(const std::vector<std::string> &topologyPaths) {
+        this->topologyPaths = topologyPaths;
+    }
+
+private:
+    bool corrupt;
+    int64_t length;
+    int64_t offset;                  // Offset of the block in the file
+    std::vector<std::string> hosts;  // Datanode hostnames
+    std::vector<std::string> names;  // Datanode IP:xferPort for getting block
+    std::vector<std::string> topologyPaths;  // Full path name in network topo
+};
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_BLOCKLOCATION_H_ */

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h

@@ -26,6 +26,9 @@ namespace internal {
 
 class BlockReader {
 public:
+    BlockReader() {
+    }
+
     virtual ~BlockReader() {
     }
 
@@ -42,15 +45,18 @@ public:
      * @return return the number of bytes filled in the buffer,
      *  it may less than size. Return 0 if reach the end of block.
      */
-    virtual int32_t read(char * buf, int32_t size) = 0;
+    virtual int32_t read(char *buf, int32_t size) = 0;
 
     /**
      * Move the cursor forward len bytes.
      * @param len The number of bytes to skip.
      */
     virtual void skip(int64_t len) = 0;
-};
 
+private:
+    BlockReader(const BlockReader &other);
+    BlockReader &operator=(const BlockReader &other);
+};
 }
 }
 

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.cc

@@ -0,0 +1,87 @@
+/**
+ * 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 "DirectoryIterator.h"
+#include "FileStatus.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "FileSystemImpl.h"
+#include "StatusInternal.h"
+
+using namespace hdfs::internal;
+
+namespace hdfs {
+
+DirectoryIterator::DirectoryIterator()
+    : needLocations(false), hasNextItem(false), filesystem(NULL), next(0) {
+}
+
+DirectoryIterator::DirectoryIterator(hdfs::internal::FileSystemImpl *const fs,
+                                     const std::string &path,
+                                     bool needLocations)
+    : needLocations(needLocations),
+      hasNextItem(false),
+      filesystem(fs),
+      next(0),
+      path(path) {
+}
+
+bool DirectoryIterator::getListing() {
+    bool more;
+
+    if (NULL == filesystem) {
+        return false;
+    }
+
+    next = 0;
+    lists.clear();
+    more = filesystem->getListing(path, startAfter, needLocations, lists);
+
+    if (!lists.empty()) {
+        startAfter = lists.back().getPath();
+    }
+
+    return hasNextItem = (more || !lists.empty());
+}
+
+bool DirectoryIterator::hasNext() {
+    return hasNextItem;
+}
+
+Status DirectoryIterator::getNext(FileStatus *output) {
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        if (next >= lists.size()) {
+            if (!getListing()) {
+                return Status(EOVERFLOW, "End of the dir flow");
+            }
+        }
+
+        *output = lists[next++];
+
+        if (next >= lists.size()) {
+            getListing();
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+}

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.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 _HDFS_LIBHFDS3_CLIENT_DIRECTORY_ITERATOR_H_
+#define _HDFS_LIBHFDS3_CLIENT_DIRECTORY_ITERATOR_H_
+
+#include "FileStatus.h"
+#include "Status.h"
+
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+class FileSystemImpl;
+}
+
+class DirectoryIterator {
+public:
+    DirectoryIterator();
+    bool hasNext();
+    Status getNext(FileStatus *output);
+
+private:
+    DirectoryIterator(hdfs::internal::FileSystemImpl *const fs,
+                      const std::string &path, bool needLocations);
+    bool getListing();
+
+    bool needLocations;
+    bool hasNextItem;
+    hdfs::internal::FileSystemImpl *filesystem;
+    size_t next;
+    std::string path;
+    std::string startAfter;
+    std::vector<FileStatus> lists;
+
+    friend hdfs::internal::FileSystemImpl;
+};
+}
+
+#endif /* _HDFS_LIBHFDS3_CLIENT_DIRECTORY_ITERATOR_H_ */

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

@@ -0,0 +1,518 @@
+/**
+ * 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 "DirectoryIterator.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "FileSystem.h"
+#include "FileSystemImpl.h"
+#include "FileSystemKey.h"
+#include "Hash.h"
+#include "SessionConfig.h"
+#include "StatusInternal.h"
+#include "Thread.h"
+#include "UnorderedMap.h"
+#include "WritableUtils.h"
+
+#include <algorithm>
+#include <string>
+#include <krb5/krb5.h>
+
+using namespace hdfs::internal;
+
+namespace hdfs {
+
+namespace internal {
+
+static std::string ExtractPrincipalFromTicketCache(
+    const std::string &cachePath) {
+    krb5_context cxt = NULL;
+    krb5_ccache ccache = NULL;
+    krb5_principal principal = NULL;
+    krb5_error_code ec = 0;
+    std::string errmsg, retval;
+    char *priName = NULL;
+
+    if (!cachePath.empty()) {
+        if (0 != setenv("KRB5CCNAME", cachePath.c_str(), 1)) {
+            THROW(HdfsIOException, "Cannot set env parameter \"KRB5CCNAME\"");
+        }
+    }
+
+    do {
+        if (0 != (ec = krb5_init_context(&cxt))) {
+            break;
+        }
+
+        if (0 != (ec = krb5_cc_default(cxt, &ccache))) {
+            break;
+        }
+
+        if (0 != (ec = krb5_cc_get_principal(cxt, ccache, &principal))) {
+            break;
+        }
+
+        if (0 != (ec = krb5_unparse_name(cxt, principal, &priName))) {
+            break;
+        }
+    } while (0);
+
+    if (!ec) {
+        retval = priName;
+    } else {
+        if (cxt) {
+            errmsg = krb5_get_error_message(cxt, ec);
+        } else {
+            errmsg = "Cannot initialize kerberos context";
+        }
+    }
+
+    if (priName != NULL) {
+        krb5_free_unparsed_name(cxt, priName);
+    }
+
+    if (principal != NULL) {
+        krb5_free_principal(cxt, principal);
+    }
+
+    if (ccache != NULL) {
+        krb5_cc_close(cxt, ccache);
+    }
+
+    if (cxt != NULL) {
+        krb5_free_context(cxt);
+    }
+
+    if (!errmsg.empty()) {
+        THROW(HdfsIOException,
+              "FileSystem: Filed to extract principal from ticket cache: %s",
+              errmsg.c_str());
+    }
+
+    return retval;
+}
+
+static std::string ExtractPrincipalFromToken(const Token &token) {
+    std::string realUser, owner;
+    std::string identifier = token.getIdentifier();
+    WritableUtils cin(&identifier[0], identifier.size());
+    char version;
+
+    try {
+        version = cin.readByte();
+
+        if (version != 0) {
+            THROW(HdfsIOException, "Unknown version of delegation token");
+        }
+
+        owner = cin.ReadText();
+        cin.ReadText();
+        realUser = cin.ReadText();
+        return realUser.empty() ? owner : realUser;
+    } catch (const std::range_error &e) {
+    }
+
+    THROW(HdfsIOException, "Cannot extract principal from token");
+}
+}
+
+FileSystem::FileSystem(const Config &conf) : conf(conf) {
+}
+
+FileSystem::~FileSystem() {
+    impl.reset();
+}
+
+Status FileSystem::connect() {
+    try {
+        internal::SessionConfig sconf(*conf.impl);
+        return connect(sconf.getDefaultUri().c_str(), NULL, NULL);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::connect(const std::string &uri) {
+    try {
+        connect(uri, NULL, NULL);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+static shared_ptr<FileSystemImpl> ConnectInternal(const std::string &uri,
+                                                  const std::string &principal,
+                                                  const Token *token,
+                                                  Config &conf) {
+    if (uri.empty()) {
+        THROW(InvalidParameter, "Invalid HDFS uri.");
+    }
+
+    FileSystemKey key(uri, principal.c_str());
+
+    if (token) {
+        key.addToken(*token);
+    }
+
+    return shared_ptr<FileSystemImpl>(new FileSystemImpl(key, conf));
+}
+
+Status FileSystem::connect(const std::string &uri, const std::string &username,
+                           const std::string &token) {
+    AuthMethod auth;
+    std::string principal;
+
+    THROW(HdfsIOException, "FileSystem: already connected.");
+
+    try {
+        SessionConfig sconf(*conf.impl);
+        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->connect();
+            return Status::OK();
+        } else if (!username.empty()) {
+            principal = username;
+        }
+
+        if (auth == AuthMethod::KERBEROS) {
+            principal =
+                ExtractPrincipalFromTicketCache(sconf.getKerberosCachePath());
+        }
+
+        impl = ConnectInternal(uri, principal, NULL, conf);
+        impl->connect();
+    } catch (...) {
+        impl.reset();
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+void FileSystem::disconnect() {
+    impl.reset();
+}
+
+Status FileSystem::getDefaultReplication(int *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getDefaultReplication();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getDefaultBlockSize(int64_t *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getDefaultBlockSize();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getHomeDirectory(std::string *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getHomeDirectory();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::deletePath(const std::string &path, bool recursive) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->deletePath(path.c_str(), recursive)) {
+            return Status(EIO);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::mkdir(const std::string &path,
+                         const Permission &permission) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->mkdir(path.c_str(), permission)) {
+            return Status(EIO);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::mkdirs(const std::string &path,
+                          const Permission &permission) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->mkdirs(path.c_str(), permission)) {
+            return Status(EIO);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getFileStatus(const std::string &path,
+                                 FileStatus *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getFileStatus(path.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getFileBlockLocations(const std::string &path, int64_t start,
+                                         int64_t len,
+                                         std::vector<BlockLocation> *output) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getFileBlockLocations(path.c_str(), start, len);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::listDirectory(const std::string &path,
+                                 DirectoryIterator *output) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->listDirectory(path.c_str(), false);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::setOwner(const std::string &path,
+                            const std::string &username,
+                            const std::string &groupname) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        impl->setOwner(path.c_str(), username.c_str(), groupname.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::setTimes(const std::string &path, int64_t mtime,
+                            int64_t atime) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        impl->setTimes(path.c_str(), mtime, atime);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::setPermission(const std::string &path,
+                                 const Permission &permission) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        impl->setPermission(path.c_str(), permission);
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::setReplication(const std::string &path, short replication) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->setReplication(path.c_str(), replication)) {
+            return Status(EIO);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::rename(const std::string &src, const std::string &dst) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->rename(src.c_str(), dst.c_str())) {
+            return Status(EIO);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::setWorkingDirectory(const std::string &path) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        impl->setWorkingDirectory(path.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getWorkingDirectory(std::string *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getWorkingDirectory();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::exist(const std::string &path) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        if (false == impl->exist(path.c_str())) {
+            return Status(ENOENT);
+        }
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getStats(FileSystemStats *output) const {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getFsStats();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getDelegationToken(const std::string &renewer,
+                                      std::string *output) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getDelegationToken(renewer.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::getDelegationToken(std::string *output) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->getDelegationToken();
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::renewDelegationToken(const std::string &token,
+                                        int64_t *output) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        *output = impl->renewDelegationToken(token.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+
+Status FileSystem::cancelDelegationToken(const std::string &token) {
+    CHECK_PARAMETER(impl, EIO, "FileSystem: not connected.");
+
+    try {
+        impl->cancelDelegationToken(token.c_str());
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
+    }
+
+    return Status::OK();
+}
+}

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

@@ -0,0 +1,277 @@
+/**
+ * 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_CLIENT_FILESYSTEM_H_
+#define _HDFS_LIBHDFS3_CLIENT_FILESYSTEM_H_
+
+#include "BlockLocation.h"
+#include "Config.h"
+#include "DirectoryIterator.h"
+#include "FileStatus.h"
+#include "FileSystemStats.h"
+#include "Permission.h"
+#include "SharedPtr.h"
+#include "Status.h"
+#include "Token.h"
+
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+struct FileSystemImpl;
+}
+
+class FileSystem {
+public:
+    /**
+     * Construct a FileSystem
+     * @param conf hdfs configuration
+     */
+    FileSystem(const Config &conf);
+
+    /**
+     * Destroy a HdfsFileSystem instance
+     */
+    ~FileSystem();
+
+    /**
+     * Connect to default hdfs.
+     * @return the result status of this operation
+     */
+    Status connect();
+
+    /**
+     * Connect to hdfs
+     * @param uri hdfs connection uri, hdfs://host:port
+     * @return the result status of this operation
+     */
+    Status connect(const std::string &uri);
+
+    /**
+     * Connect to hdfs with user or token
+     * 	username and token cannot be set at the same time
+     * @param uri connection uri.
+     * @param username user used to connect to hdfs
+     * @param token token used to connect to hdfs
+     * @return the result status of this operation
+     */
+    Status connect(const std::string &uri, const std::string &username,
+                   const std::string &token);
+
+    /**
+     * disconnect from hdfs
+     */
+    void disconnect();
+
+    /**
+     * To get default number of replication.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getDefaultReplication(int *output) const;
+
+    /**
+     * To get the default number of block size.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getDefaultBlockSize(int64_t *output) const;
+
+    /**
+     * To get the home directory.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getHomeDirectory(std::string *output) const;
+
+    /**
+     * To delete a file or directory.
+     * @param path the path to be deleted.
+     * @param recursive if path is a directory, delete the contents recursively.
+     * @return the result status of this operation
+     */
+    Status deletePath(const std::string &path, bool recursive);
+
+    /**
+     * To create a directory which given permission.
+     * @param path the directory path which is to be created.
+     * @param permission directory permission.
+     * @return the result status of this operation
+     */
+    Status mkdir(const std::string &path, const Permission &permission);
+
+    /**
+     * To create a directory which given permission.
+     * If parent path does not exits, create it.
+     * @param path the directory path which is to be created.
+     * @param permission directory permission.
+     * @return the result status of this operation
+     */
+    Status mkdirs(const std::string &path, const Permission &permission);
+
+    /**
+     * To get path information.
+     * @param path the path which information is to be returned.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getFileStatus(const std::string &path, FileStatus *output) const;
+
+    /**
+     * Return an array containing hostnames, offset and size of
+     * portions of the given file.
+     *
+     * This call is most helpful with DFS, where it returns
+     * hostnames of machines that contain the given file.
+     *
+     * The FileSystem will simply return an elt containing 'localhost'.
+     *
+     * @param path path is used to identify an FS since an FS could have
+     *          another FS that it could be delegating the call to
+     * @param start offset into the given file
+     * @param len length for which to get locations for
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getFileBlockLocations(const std::string &path, int64_t start,
+                                 int64_t len,
+                                 std::vector<BlockLocation> *output);
+
+    /**
+     * list the contents of a directory.
+     * @param path The directory path.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status listDirectory(const std::string &path, DirectoryIterator *output);
+
+    /**
+     * To set the owner and the group of the path.
+     * username and groupname cannot be empty at the same time.
+     * @param path the path which owner of group is to be changed.
+     * @param username new user name.
+     * @param groupname new group.
+     * @return the result status of this operation
+     */
+    Status setOwner(const std::string &path, const std::string &username,
+                    const std::string &groupname);
+
+    /**
+     * To set the access time or modification time of a path.
+     * @param path the path which access time or modification time is to be
+     * changed.
+     * @param mtime new modification time.
+     * @param atime new access time.
+     * @return the result status of this operation
+     */
+    Status setTimes(const std::string &path, int64_t mtime, int64_t atime);
+
+    /**
+     * To set the permission of a path.
+     * @param path the path which permission is to be changed.
+     * @param permission new permission.
+     * @return the result status of this operation
+     */
+    Status setPermission(const std::string &path, const Permission &permission);
+
+    /**
+     * To set the number of replication.
+     * @param path the path which number of replication is to be changed.
+     * @param replication new number of replication.
+     * @return return true if success.
+     * @return the result status of this operation
+     */
+    Status setReplication(const std::string &path, short replication);
+
+    /**
+     * To rename a path.
+     * @param src old path.
+     * @param dst new path.
+     * @return the result status of this operation
+     */
+    Status rename(const std::string &src, const std::string &dst);
+
+    /**
+     * To set working directory.
+     * @param path new working directory.
+     * @return the result status of this operation
+     */
+    Status setWorkingDirectory(const std::string &path);
+
+    /**
+     * To get working directory.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getWorkingDirectory(std::string *output) const;
+
+    /**
+     * To test if the path exist.
+     * @param path the path which is to be tested.
+     * @return the result status of this operation
+     */
+    Status exist(const std::string &path) const;
+
+    /**
+     * To get the file system status.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getStats(FileSystemStats *output) const;
+
+    /**
+     * Get a valid Delegation Token.
+     * @param renewer the designated renewer for the token
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getDelegationToken(const std::string &renewer, std::string *output);
+
+    /**
+     * Get a valid Delegation Token using the default user as renewer.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getDelegationToken(std::string *output);
+
+    /**
+     * Renew an existing delegation token.
+     * @param token delegation token obtained earlier
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status renewDelegationToken(const std::string &token, int64_t *output);
+
+    /**
+     * Cancel an existing delegation token.
+     * @param token delegation token
+     * @return the result status of this operation
+     */
+    Status cancelDelegationToken(const std::string &token);
+
+private:
+    FileSystem(const FileSystem &other);
+    FileSystem &operator=(const FileSystem &other);
+    Config conf;
+    hdfs::internal::shared_ptr<internal::FileSystemImpl> impl;
+
+    friend class InputStream;
+    friend class OutputStream;
+};
+}
+#endif /* _HDFS_LIBHDFS3_CLIENT_FILESYSTEM_H_ */

+ 760 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc

@@ -0,0 +1,760 @@
+/**
+ * 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 "Atomic.h"
+#include "BlockLocation.h"
+#include "DirectoryIterator.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "FileStatus.h"
+#include "FileSystemImpl.h"
+#include "FileSystemStats.h"
+#include "InputStream.h"
+#include "Logger.h"
+#include "StringUtil.h"
+#include "server/LocatedBlocks.h"
+#include "server/NamenodeInfo.h"
+#include "server/NamenodeProxy.h"
+
+#include <cstring>
+#include <deque>
+#include <inttypes.h>
+#include <libxml/uri.h>
+#include <strings.h>
+
+using std::string;
+using std::deque;
+using std::vector;
+
+namespace hdfs {
+namespace internal {
+
+static const std::string GetAbsPath(const std::string &prefix,
+                                    const std::string &path) {
+    if (path.empty()) {
+        return prefix;
+    }
+
+    if ('/' == path[0]) {
+        return path;
+    } else {
+        return prefix + "/" + path;
+    }
+}
+
+/*
+ * Return the canonical absolute name of file NAME.
+ * A canonical name does not contain any `.', `..' components nor any repeated
+ * path separators ('/')
+ */
+static const std::string CanonicalizePath(const std::string &path) {
+    int skip = 0;
+    string retval;
+    vector<string> components = StringSplit(path, "/");
+    deque<string> tmp;
+    vector<string>::reverse_iterator s = components.rbegin();
+
+    while (s != components.rend()) {
+        if (s->empty() || *s == ".") {
+            ++s;
+        } else if (*s == "..") {
+            ++skip;
+            ++s;
+        } else {
+            if (skip <= 0) {
+                tmp.push_front(*s);
+            } else {
+                --skip;
+            }
+
+            ++s;
+        }
+    }
+
+    for (size_t i = 0; i < tmp.size(); ++i) {
+        retval += "/";
+        retval += tmp[i];
+    }
+
+    return retval.empty() ? "/" : retval;
+}
+
+FileSystemImpl::FileSystemImpl(const FileSystemKey &key, const Config &c)
+    : conf(c),
+      key(key),
+      openedOutputStream(0),
+      nn(NULL),
+      sconf(*c.impl),
+      user(key.getUser()) {
+    static atomic<uint32_t> count(0);
+    std::stringstream ss;
+    srand((unsigned int)time(NULL));
+    ss << "libhdfs3_client_random_" << rand() << "_count_" << ++count << "_pid_"
+       << getpid() << "_tid_" << pthread_self();
+    clientName = ss.str();
+    workingDir = std::string("/user/") + user.getEffectiveUser();
+#ifdef MOCK
+    stub = NULL;
+#endif
+    // set log level
+    RootLogger.setLogSeverity(sconf.getLogSeverity());
+}
+
+/**
+ * Destroy a FileSystemBase instance
+ */
+FileSystemImpl::~FileSystemImpl() {
+    try {
+        disconnect();
+    } catch (...) {
+    }
+}
+
+const std::string FileSystemImpl::getStandardPath(const char *path) {
+    std::string base;
+    {
+        lock_guard<mutex> lock(mutWorkingDir);
+        base = workingDir;
+    }
+    return CanonicalizePath(GetAbsPath(base, path));
+}
+
+const char *FileSystemImpl::getClientName() {
+    return clientName.c_str();
+}
+
+void FileSystemImpl::connect() {
+    std::string host, port, uri;
+    std::vector<NamenodeInfo> namenodeInfos;
+
+    if (nn) {
+        THROW(HdfsIOException, "FileSystemImpl: already connected.");
+    }
+
+    host = key.getHost();
+    port = key.getPort();
+    uri += key.getScheme() + "://" + host;
+
+    if (port.empty()) {
+        try {
+            Status status = NamenodeInfo::GetHANamenodeInfo(key.getHost(), conf,
+                                                            &namenodeInfos);
+        } catch (const HdfsConfigNotFound &e) {
+            NESTED_THROW(InvalidParameter,
+                         "Cannot parse URI: %s, missing port or invalid HA "
+                         "configuration",
+                         uri.c_str());
+        }
+
+        tokenService = "ha-hdfs:";
+        tokenService += host;
+    } else {
+        std::stringstream ss;
+        ss << host << ":" << port;
+        namenodeInfos.resize(1);
+        namenodeInfos[0].setRpcAddr(ss.str());
+        tokenService = namenodeInfos[0].getRpcAddr();
+    }
+
+#ifdef MOCK
+    nn = stub->getNamenode();
+#else
+    nn = new NamenodeProxy(
+        namenodeInfos, tokenService, sconf,
+        RpcAuth(user, RpcAuth::ParseMethod(sconf.getRpcAuthMethod())));
+#endif
+    /*
+     * To test if the connection is ok
+     */
+    getFsStats();
+}
+
+/**
+ * disconnect from hdfs
+ */
+void FileSystemImpl::disconnect() {
+    if (nn) {
+        nn->close();
+        delete nn;
+    }
+
+    nn = NULL;
+}
+
+/**
+ * To get default number of replication.
+ * @return the default number of replication.
+ */
+int FileSystemImpl::getDefaultReplication() const {
+    return sconf.getDefaultReplica();
+}
+
+/**
+ * To get the default number of block size.
+ * @return the default block size.
+ */
+int64_t FileSystemImpl::getDefaultBlockSize() const {
+    return sconf.getDefaultBlockSize();
+}
+
+/**
+ * To get the home directory.
+ * @return home directory.
+ */
+std::string FileSystemImpl::getHomeDirectory() const {
+    return std::string("/user/") + user.getEffectiveUser();
+}
+
+/**
+ * To delete a file or directory.
+ * @param path the path to be deleted.
+ * @param recursive if path is a directory, delete the contents recursively.
+ * @return return true if success.
+ */
+
+bool FileSystemImpl::deletePath(const char *path, bool recursive) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return nn->deleteFile(getStandardPath(path), recursive);
+}
+
+/**
+ * To create a directory which given permission.
+ * @param path the directory path which is to be created.
+ * @param permission directory permission.
+ * @return return true if success.
+ */
+
+bool FileSystemImpl::mkdir(const char *path, const Permission &permission) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return nn->mkdirs(getStandardPath(path), permission, false);
+}
+
+/**
+ * To create a directory which given permission.
+ * If parent path does not exits, create it.
+ * @param path the directory path which is to be created.
+ * @param permission directory permission.
+ * @return return true if success.
+ */
+
+bool FileSystemImpl::mkdirs(const char *path, const Permission &permission) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return nn->mkdirs(getStandardPath(path), permission, true);
+}
+
+/**
+ * To get path information.
+ * @param path the path which information is to be returned.
+ * @return the path information.
+ */
+FileStatus FileSystemImpl::getFileStatus(const char *path) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return nn->getFileInfo(getStandardPath(path));
+}
+
+static void Convert(BlockLocation &bl, const LocatedBlock &lb) {
+    const std::vector<DatanodeInfo> &nodes = lb.getLocations();
+    bl.setCorrupt(lb.isCorrupt());
+    bl.setLength(lb.getNumBytes());
+    bl.setOffset(lb.getOffset());
+    std::vector<std::string> hosts(nodes.size());
+    std::vector<std::string> names(nodes.size());
+    std::vector<std::string> topologyPaths(nodes.size());
+
+    for (size_t i = 0; i < nodes.size(); ++i) {
+        hosts[i] = nodes[i].getHostName();
+        names[i] = nodes[i].getXferAddr();
+        topologyPaths[i] =
+            nodes[i].getLocation() + '/' + nodes[i].getXferAddr();
+    }
+
+    bl.setNames(names);
+    bl.setHosts(hosts);
+    bl.setTopologyPaths(topologyPaths);
+}
+
+std::vector<BlockLocation> FileSystemImpl::getFileBlockLocations(
+    const char *path, int64_t start, int64_t len) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    if (start < 0) {
+        THROW(InvalidParameter,
+              "Invalid input: start offset should be positive");
+    }
+
+    if (len < 0) {
+        THROW(InvalidParameter, "Invalid input: length should be positive");
+    }
+
+    LocatedBlocks lbs;
+    nn->getBlockLocations(getStandardPath(path), start, len, lbs);
+    std::vector<LocatedBlock> blocks = lbs.getBlocks();
+    std::vector<BlockLocation> retval(blocks.size());
+
+    for (size_t i = 0; i < blocks.size(); ++i) {
+        Convert(retval[i], blocks[i]);
+    }
+
+    return retval;
+}
+
+/**
+ * list the contents of a directory.
+ * @param path the directory path.
+ * @return return the path informations in the given directory.
+ */
+DirectoryIterator FileSystemImpl::listDirectory(const char *path,
+                                                bool needLocation) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return DirectoryIterator(this, getStandardPath(path), needLocation);
+}
+
+/**
+ * To set the owner and the group of the path.
+ * username and groupname cannot be empty at the same time.
+ * @param path the path which owner of group is to be changed.
+ * @param username new user name.
+ * @param groupname new group.
+ */
+void FileSystemImpl::setOwner(const char *path, const char *username,
+                              const char *groupname) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    if ((NULL == username || !strlen(username)) &&
+        (NULL == groupname || !strlen(groupname))) {
+        THROW(InvalidParameter,
+              "Invalid input: username and groupname should not be empty");
+    }
+
+    nn->setOwner(getStandardPath(path), username != NULL ? username : "",
+                 groupname != NULL ? groupname : "");
+}
+
+/**
+ * To set the access time or modification time of a path.
+ * @param path the path which access time or modification time is to be changed.
+ * @param mtime new modification time.
+ * @param atime new access time.
+ */
+void FileSystemImpl::setTimes(const char *path, int64_t mtime, int64_t atime) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    nn->setTimes(getStandardPath(path), mtime, atime);
+}
+
+/**
+ * To set the permission of a path.
+ * @param path the path which permission is to be changed.
+ * @param permission new permission.
+ */
+void FileSystemImpl::setPermission(const char *path,
+                                   const Permission &permission) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    nn->setPermission(getStandardPath(path), permission);
+}
+
+/**
+ * To set the number of replication.
+ * @param path the path which number of replication is to be changed.
+ * @param replication new number of replication.
+ * @return return true if success.
+ */
+
+bool FileSystemImpl::setReplication(const char *path, short replication) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    return nn->setReplication(getStandardPath(path), replication);
+}
+
+/**
+ * To rename a path.
+ * @param src old path.
+ * @param dst new path.
+ * @return return true if success.
+ */
+
+bool FileSystemImpl::rename(const char *src, const char *dst) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == src || !strlen(src)) {
+        THROW(InvalidParameter, "Invalid input: src should not be empty");
+    }
+
+    if (NULL == dst || !strlen(dst)) {
+        THROW(InvalidParameter, "Invalid input: dst should not be empty");
+    }
+
+    return nn->rename(getStandardPath(src), getStandardPath(dst));
+}
+
+/**
+ * To set working directory.
+ * @param path new working directory.
+ */
+void FileSystemImpl::setWorkingDirectory(const char *path) {
+    if (NULL == path) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    if (!strlen(path) || '/' != path[0]) {
+        THROW(InvalidParameter,
+              "Invalid input: path should be an absolute path");
+    }
+
+    lock_guard<mutex> lock(mutWorkingDir);
+    workingDir = path;
+}
+
+/**
+ * To get working directory.
+ * @return working directory.
+ */
+std::string FileSystemImpl::getWorkingDirectory() const {
+    return workingDir;
+}
+
+/**
+ * To test if the path exist.
+ * @param path the path which is to be tested.
+ * @return return true if the path exist.
+ */
+
+bool FileSystemImpl::exist(const char *path) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == path || !strlen(path)) {
+        THROW(InvalidParameter, "Invalid input: path should not be empty");
+    }
+
+    try {
+        getFileStatus(path);
+    } catch (const FileNotFoundException &e) {
+        return false;
+    }
+
+    return true;
+}
+
+/**
+ * To get the file system status.
+ * @return the file system status.
+ */
+FileSystemStats FileSystemImpl::getFsStats() {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    std::vector<int64_t> retval = nn->getFsStats();
+    assert(retval.size() >= 3);
+    return FileSystemStats(retval[0], retval[1], retval[2]);
+}
+
+static std::string ConstructTempFilePath(const std::string &path,
+                                         const std::string clientName) {
+    std::stringstream ss;
+    srand((unsigned int)time(NULL));
+    static atomic<uint32_t> count(0);
+    std::vector<std::string> components = StringSplit(path, "/");
+    ss << '/';
+
+    for (size_t i = components.size(); i > 0; --i) {
+        if (!components[i - 1].empty()) {
+            components[i - 1].clear();
+            break;
+        }
+    }
+
+    for (size_t i = 0; i < components.size(); ++i) {
+        if (!components[i].empty()) {
+            ss << components[i] << '/';
+        }
+    }
+
+    ss << "._client_" << clientName << "_random_" << rand() << "_count_"
+       << ++count << "_tid_" << pthread_self() << "_TRUNCATE_TMP";
+    return ss.str();
+}
+
+std::string FileSystemImpl::getDelegationToken(const char *renewer) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    if (NULL == renewer || !strlen(renewer)) {
+        THROW(InvalidParameter, "Invalid input: renewer should not be empty.");
+    }
+
+    Token retval = nn->getDelegationToken(renewer);
+    retval.setService(tokenService);
+    return retval.toString();
+}
+
+std::string FileSystemImpl::getDelegationToken() {
+    return getDelegationToken(key.getUser().getPrincipal().c_str());
+}
+
+int64_t FileSystemImpl::renewDelegationToken(const std::string &token) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    Token t;
+    t.fromString(token);
+    return nn->renewDelegationToken(t);
+}
+
+void FileSystemImpl::cancelDelegationToken(const std::string &token) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    Token t;
+    t.fromString(token);
+    nn->cancelDelegationToken(t);
+}
+
+void FileSystemImpl::getBlockLocations(const std::string &src, int64_t offset,
+                                       int64_t length, LocatedBlocks &lbs) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->getBlockLocations(src, offset, length, lbs);
+}
+
+void FileSystemImpl::create(const std::string &src, const Permission &masked,
+                            int flag, bool createParent, short replication,
+                            int64_t blockSize) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->create(src, masked, clientName, flag, createParent, replication,
+               blockSize);
+}
+
+shared_ptr<LocatedBlock> FileSystemImpl::append(const std::string &src) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->append(src, clientName);
+}
+
+void FileSystemImpl::abandonBlock(const ExtendedBlock &b,
+                                  const std::string &src) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->abandonBlock(b, src, clientName);
+}
+
+shared_ptr<LocatedBlock> FileSystemImpl::addBlock(
+    const std::string &src, const ExtendedBlock *previous,
+    const std::vector<DatanodeInfo> &excludeNodes) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->addBlock(src, clientName, previous, excludeNodes);
+}
+
+shared_ptr<LocatedBlock> FileSystemImpl::getAdditionalDatanode(
+    const std::string &src, const ExtendedBlock &blk,
+    const std::vector<DatanodeInfo> &existings,
+    const std::vector<std::string> &storageIDs,
+    const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->getAdditionalDatanode(src, blk, existings, storageIDs, excludes,
+                                     numAdditionalNodes, clientName);
+}
+
+bool FileSystemImpl::complete(const std::string &src,
+                              const ExtendedBlock *last) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->complete(src, clientName, last);
+}
+
+/*void FileSystemImpl::reportBadBlocks(const std::vector<LocatedBlock> &blocks)
+{
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->reportBadBlocks(blocks);
+}*/
+
+void FileSystemImpl::fsync(const std::string &src) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->fsync(src, clientName);
+}
+
+shared_ptr<LocatedBlock> FileSystemImpl::updateBlockForPipeline(
+    const ExtendedBlock &block) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->updateBlockForPipeline(block, clientName);
+}
+
+void FileSystemImpl::updatePipeline(
+    const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
+    const std::vector<DatanodeInfo> &newNodes,
+    const std::vector<std::string> &storageIDs) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    nn->updatePipeline(clientName, oldBlock, newBlock, newNodes, storageIDs);
+}
+
+bool FileSystemImpl::getListing(const std::string &src,
+                                const std::string &startAfter,
+                                bool needLocation,
+                                std::vector<FileStatus> &dl) {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    return nn->getListing(src, startAfter, needLocation, dl);
+}
+
+bool FileSystemImpl::renewLease() {
+    if (!nn) {
+        THROW(HdfsIOException, "FileSystemImpl: not connected.");
+    }
+
+    // protected by LeaseRenewer's lock
+    if (0 == openedOutputStream) {
+        return false;
+    }
+
+    try {
+        nn->renewLease(clientName);
+        return true;
+    } catch (const HdfsException &e) {
+        LOG(LOG_ERROR,
+            "Failed to renew lease for filesystem which client name "
+            "is %s, since:\n%s",
+            getClientName(), GetExceptionDetail(e));
+    } catch (const std::exception &e) {
+        LOG(LOG_ERROR,
+            "Failed to renew lease for filesystem which client name is "
+            "%s, since:\n%s",
+            getClientName(), e.what());
+    }
+
+    return false;
+}
+
+void FileSystemImpl::registerOpenedOutputStream() {
+    // protected by LeaseRenewer's lock
+    ++openedOutputStream;
+}
+
+bool FileSystemImpl::unregisterOpenedOutputStream() {
+    // protected by LeaseRenewer's lock
+    if (openedOutputStream > 0) {
+        --openedOutputStream;
+    }
+
+    return openedOutputStream == 0;
+}
+}
+}

+ 478 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.h

@@ -0,0 +1,478 @@
+/**
+ * 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_CLIENT_FILESYSTEMIMPL_H_
+#define _HDFS_LIBHDFS3_CLIENT_FILESYSTEMIMPL_H_
+
+#include "BlockLocation.h"
+#include "Config.h"
+#include "DirectoryIterator.h"
+#include "FileStatus.h"
+#include "FileSystemKey.h"
+#include "FileSystemStats.h"
+#include "server/LocatedBlocks.h"
+#include "server/Namenode.h"
+#include "SessionConfig.h"
+#include "UnorderedMap.h"
+#include "UserInfo.h"
+#ifdef MOCK
+#include "NamenodeStub.h"
+#endif
+
+#include <string>
+#include <vector>
+
+namespace hdfs {
+class Permission;
+}
+
+namespace hdfs {
+namespace internal {
+
+class FileSystemImpl {
+public:
+    /**
+     * Construct a FileSystemImpl instance.
+     * @param key a key which can be uniquely identify a FileSystemImpl
+     * instance.
+     * @param c a configuration objecto used to initialize the instance.
+     */
+    FileSystemImpl(const FileSystemKey &key, const Config &c);
+
+    /**
+     * Destroy a FileSystemBase instance
+     */
+    ~FileSystemImpl();
+
+    /**
+     * Format the path to a absolute canonicalized path.
+     * @param path target path to be hendled.
+     * @return return a absolute canonicalized path.
+     */
+    const std::string getStandardPath(const char *path);
+
+    /**
+     * To get the client unique ID.
+     * @return return the client unique ID.
+     */
+    const char *getClientName();
+
+    /**
+     * Connect to hdfs
+     */
+    void connect();
+
+    /**
+     * disconnect from hdfs
+     */
+    void disconnect();
+
+    /**
+     * To get default number of replication.
+     * @return the default number of replication.
+     */
+    int getDefaultReplication() const;
+
+    /**
+     * To get the default block size.
+     * @return the default block size.
+     */
+    int64_t getDefaultBlockSize() const;
+
+    /**
+     * To get the home directory.
+     * @return home directory.
+     */
+    std::string getHomeDirectory() const;
+
+    /**
+     * To delete a file or directory.
+     * @param path the path to be deleted.
+     * @param recursive if path is a directory, delete the contents recursively.
+     * @return return true if success.
+     */
+    bool deletePath(const char *path, bool recursive);
+
+    /**
+     * To create a directory with given permission.
+     * @param path the directory path which is to be created.
+     * @param permission directory permission.
+     * @return return true if success.
+     */
+    bool mkdir(const char *path, const Permission &permission);
+
+    /**
+     * To create a directory which given permission.
+     * If parent path does not exits, create it.
+     * @param path the directory path which is to be created.
+     * @param permission directory permission.
+     * @return return true if success.
+     */
+    bool mkdirs(const char *path, const Permission &permission);
+
+    /**
+     * To get path information.
+     * @param path the path which information is to be returned.
+     * @return the path information.
+     */
+    FileStatus getFileStatus(const char *path);
+
+    /**
+     * Return an array containing hostnames, offset and size of
+     * portions of the given file.
+     *
+     * This call is most helpful with DFS, where it returns
+     * hostnames of machines that contain the given file.
+     *
+     * The FileSystem will simply return an elt containing 'localhost'.
+     *
+     * @param path path is used to identify an FS since an FS could have
+     *          another FS that it could be delegating the call to
+     * @param start offset into the given file
+     * @param len length for which to get locations for
+     */
+    std::vector<BlockLocation> getFileBlockLocations(const char *path,
+                                                     int64_t start,
+                                                     int64_t len);
+
+    /**
+     * list the contents of a directory.
+     * @param path the directory path.
+     * @return Return a iterator to visit all elements in this directory.
+     */
+    DirectoryIterator listDirectory(const char *path, bool needLocation);
+
+    /**
+     * To set the owner and the group of the path.
+     * username and groupname cannot be empty at the same time.
+     * @param path the path which owner of group is to be changed.
+     * @param username new user name.
+     * @param groupname new group.
+     */
+    void setOwner(const char *path, const char *username,
+                  const char *groupname);
+
+    /**
+     * To set the access time or modification time of a path.
+     * @param path the path which access time or modification time is to be
+     * changed.
+     * @param mtime new modification time.
+     * @param atime new access time.
+     */
+    void setTimes(const char *path, int64_t mtime, int64_t atime);
+
+    /**
+     * To set the permission of a path.
+     * @param path the path which permission is to be changed.
+     * @param permission new permission.
+     */
+    void setPermission(const char *path, const Permission &permission);
+
+    /**
+     * To set the number of replication.
+     * @param path the path which number of replication is to be changed.
+     * @param replication new number of replication.
+     * @return return true if success.
+     */
+    bool setReplication(const char *path, short replication);
+
+    /**
+     * To rename a path.
+     * @param src old path.
+     * @param dst new path.
+     * @return return true if success.
+     */
+    bool rename(const char *src, const char *dst);
+
+    /**
+     * To set working directory.
+     * @param path new working directory.
+     */
+    void setWorkingDirectory(const char *path);
+
+    /**
+     * To get working directory.
+     * @return working directory.
+     */
+    std::string getWorkingDirectory() const;
+
+    /**
+     * To test if the path exist.
+     * @param path the path which is to be tested.
+     * @return return true if the path exist.
+     */
+    bool exist(const char *path);
+
+    /**
+     * To get the file system status.
+     * @return the file system status.
+     */
+    FileSystemStats getFsStats();
+
+    /**
+     * Get a valid Delegation Token.
+     *
+     * @param renewer the designated renewer for the token
+     * @return Token
+     * @throws IOException
+     */
+    std::string getDelegationToken(const char *renewer);
+
+    /**
+     * Get a valid Delegation Token using default user as renewer.
+     *
+     * @return Token
+     * @throws IOException
+     */
+    std::string getDelegationToken();
+
+    /**
+     * Renew an existing delegation token.
+     *
+     * @param token delegation token obtained earlier
+     * @return the new expiration time
+     * @throws IOException
+     */
+    int64_t renewDelegationToken(const std::string &token);
+
+    /**
+     * Cancel an existing delegation token.
+     *
+     * @param token delegation token
+     * @throws IOException
+     */
+    void cancelDelegationToken(const std::string &token);
+
+    /**
+     * Get locations of the blocks of the specified file within the specified
+     *range.
+     * DataNode locations for each block are sorted by
+     * the proximity to the client.
+     *
+     * The client will then have to contact
+     * one of the indicated DataNodes to obtain the actual data.
+     *
+     * @param src file name
+     * @param offset range start offset
+     * @param length range length
+     * @param lbs output the returned blocks
+     */
+    void getBlockLocations(const std::string &src, int64_t offset,
+                           int64_t length, LocatedBlocks &lbs);
+
+    /**
+     * Create a new file entry in the namespace.
+     *
+     * @param src path of the file being created.
+     * @param masked masked permission.
+     * @param flag indicates whether the file should be
+     *  overwritten if it already exists or create if it does not exist or
+     *append.
+     * @param createParent create missing parent directory if true
+     * @param replication block replication factor.
+     * @param blockSize maximum block size.
+     */
+    void create(const std::string &src, const Permission &masked, int flag,
+                bool createParent, short replication, int64_t blockSize);
+
+    /**
+     * Append to the end of the file.
+     *
+     * @param src path of the file being created.
+     * @return return the last partial block if any
+     */
+    shared_ptr<LocatedBlock> append(const std::string &src);
+
+    /**
+     * The client can give up on a block by calling abandonBlock().
+     * The client can then either obtain a new block, or complete or abandon the
+     *file.
+     * Any partial writes to the block will be discarded.
+     *
+     * @param b the block to be abandoned.
+     * @param src the file which the block belongs to.
+     */
+    void abandonBlock(const ExtendedBlock &b, const std::string &src);
+
+    /**
+     * A client that wants to write an additional block to the
+     * indicated filename (which must currently be open for writing)
+     * should call addBlock().
+     *
+     * addBlock() allocates a new block and datanodes the block data
+     * should be replicated to.
+     *
+     * addBlock() also commits the previous block by reporting
+     * to the name-node the actual generation stamp and the length
+     * of the block that the client has transmitted to data-nodes.
+     *
+     * @param src the file being created
+     * @param previous  previous block
+     * @param excludeNodes a list of nodes that should not be allocated for the
+     *current block.
+     * @return return the new block.
+     */
+    shared_ptr<LocatedBlock> addBlock(
+        const std::string &src, const ExtendedBlock *previous,
+        const std::vector<DatanodeInfo> &excludeNodes);
+
+    /**
+     * Get a datanode for an existing pipeline.
+     *
+     * @param src the file being written
+     * @param blk the block being written
+     * @param existings the existing nodes in the pipeline
+     * @param excludes the excluded nodes
+     * @param numAdditionalNodes number of additional datanodes
+     * @return return a new block information which contains new datanode.
+     */
+    shared_ptr<LocatedBlock> getAdditionalDatanode(
+        const std::string &src, const ExtendedBlock &blk,
+        const std::vector<DatanodeInfo> &existings,
+        const std::vector<std::string> &storageIDs,
+        const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes);
+
+    /**
+     * The client is done writing data to the given filename, and would
+     * like to complete it.
+     *
+     * The function returns whether the file has been closed successfully.
+     * If the function returns false, the caller should try again.
+     *
+     * close() also commits the last block of file by reporting
+     * to the name-node the actual generation stamp and the length
+     * of the block that the client has transmitted to data-nodes.
+     *
+     * A call to complete() will not return true until all the file's
+     * blocks have been replicated the minimum number of times.  Thus,
+     * DataNode failures may cause a client to call complete() several
+     * times before succeeding.
+     *
+     * @param src the file being written.
+     * @param last last block to be committed.
+     * @return return false if the client should retry.
+     */
+    bool complete(const std::string &src, const ExtendedBlock *last);
+
+    /**
+     * The client wants to report corrupted blocks (blocks with specified
+     * locations on datanodes).
+     * @param blocks Array of located blocks to report
+     */
+    /*void reportBadBlocks(const std::vector<LocatedBlock> &blocks);*/
+
+    /**
+     * Write all metadata for this file into persistent storage.
+     * The file must be currently open for writing.
+     * @param src The const std::string &representation of the path
+     */
+    void fsync(const std::string &src);
+
+    /**
+     * Get a new generation stamp together with an access token for
+     * a block under construction
+     *
+     * This method is called only when a client needs to recover a failed
+     * pipeline or set up a pipeline for appending to a block.
+     *
+     * @param block a block
+     * @return return a located block with a new generation stamp and an access
+     *token
+     */
+    shared_ptr<LocatedBlock> updateBlockForPipeline(const ExtendedBlock &block);
+
+    /**
+     * Update a pipeline for a block under construction
+     *
+     * @param oldBlock the old block
+     * @param newBlock the new block containing new generation stamp and length
+     * @param newNodes datanodes in the pipeline
+     */
+    void updatePipeline(const ExtendedBlock &oldBlock,
+                        const ExtendedBlock &newBlock,
+                        const std::vector<DatanodeInfo> &newNodes,
+                        const std::vector<std::string> &storageIDs);
+
+    /**
+     * register the output stream in filespace when it is opened.
+     */
+    void registerOpenedOutputStream();
+
+    /**
+     * unregister the output stream from filespace when it is closed.
+     */
+    bool unregisterOpenedOutputStream();
+
+    /**
+     * Get the configuration used in filesystem.
+     * @return return the configuration instance.
+     */
+    const SessionConfig &getConf() const {
+        return sconf;
+    }
+
+    /**
+     * Get the user used in filesystem.
+     * @return return the user information.
+     */
+    const UserInfo &getUserInfo() const {
+        return user;
+    }
+
+    /**
+     * Get a partial listing of the indicated directory
+     *
+     * @param src the directory name
+     * @param startAfter the name to start listing after encoded in java UTF8
+     * @param needLocation if the FileStatus should contain block locations
+     * @param dl append the returned directories.
+     * @return return true if there are more items.
+     */
+    bool getListing(const std::string &src, const std::string &startAfter,
+                    bool needLocation, std::vector<FileStatus> &dl);
+
+    /**
+     * To renew the lease.
+     *
+     * @return return false if the filesystem no long needs to renew lease.
+     */
+    bool renewLease();
+
+private:
+    FileSystemImpl(const FileSystemImpl &other);
+    FileSystemImpl &operator=(const FileSystemImpl &other);
+
+    Config conf;
+    FileSystemKey key;
+    int openedOutputStream;
+    mutex mutWorkingDir;
+    Namenode *nn;
+    SessionConfig sconf;
+    std::string clientName;
+    std::string tokenService;
+    std::string workingDir;
+    UserInfo user;
+#ifdef MOCK
+private:
+    Hdfs::Mock::NamenodeStub *stub;
+#endif
+};
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_FILESYSTEMIMPL_H_ */

+ 89 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemKey.cc

@@ -0,0 +1,89 @@
+/**
+ * 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 "FileSystemKey.h"
+
+#include <algorithm>
+#include <libxml/uri.h>
+#include <sstream>
+
+namespace hdfs {
+namespace internal {
+
+FileSystemKey::FileSystemKey(const std::string &uri, const char *u) {
+    xmlURIPtr uriobj;
+    std::stringstream ss;
+    uriobj = xmlParseURI(uri.c_str());
+
+    try {
+        if (!uriobj || uriobj->server == NULL || 0 == strlen(uriobj->server)) {
+            THROW(InvalidParameter,
+                  "Invalid input: uri: %s is not a valid URI type.",
+                  uri.c_str());
+        }
+
+        host = uriobj->server;
+
+        if (NULL == uriobj->scheme || 0 == strlen(uriobj->scheme)) {
+            scheme = "hdfs";
+        } else {
+            scheme = uriobj->scheme;
+        }
+
+        if (strcasecmp(scheme.c_str(), "hdfs")) {
+            THROW(InvalidParameter,
+                  "Invalid input: uri is not a valid URI type.");
+        }
+
+        if (u && strlen(u) > 0) {
+            user = UserInfo(u);
+        } else if (NULL == uriobj->user || 0 == strlen(uriobj->user)) {
+            user = UserInfo::LocalUser();
+        } else {
+            user = UserInfo(uriobj->user);
+        }
+
+        ss << user.getEffectiveUser();
+
+        if (uriobj->port == 0) {
+            ss << "@" << uriobj->server;
+        } else {
+            std::stringstream s;
+            s << uriobj->port;
+            port = s.str();
+            ss << "@" << uriobj->server << ":" << uriobj->port;
+        }
+
+        authority = ss.str();
+    } catch (...) {
+        if (uriobj) {
+            xmlFreeURI(uriobj);
+        }
+
+        throw;
+    }
+
+    xmlFreeURI(uriobj);
+    std::transform(authority.begin(), authority.end(), authority.begin(),
+                   tolower);
+    std::transform(scheme.begin(), scheme.end(), scheme.begin(), tolower);
+}
+}
+}

+ 97 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemKey.h

@@ -0,0 +1,97 @@
+/**
+ * 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_CLIENT_FILESYSTEMKEY_H_
+#define _HDFS_LIBHDFS3_CLIENT_FILESYSTEMKEY_H_
+
+#include "Hash.h"
+#include "UserInfo.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class FileSystemKey {
+public:
+    FileSystemKey(const std::string &uri, const char *user);
+
+    FileSystemKey(const std::string &auth, const std::string &host,
+                  const std::string &port, const std::string &scheme,
+                  const std::string &user, size_t u)
+        : authority(auth), host(host), port(port), scheme(scheme), user(user) {
+    }
+
+    bool operator==(const FileSystemKey &other) const {
+        return scheme == other.scheme && authority == other.authority;
+    }
+
+    size_t hash_value() const {
+        size_t values[] = {StringHasher(scheme), StringHasher(authority)};
+        return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+    }
+
+    const std::string &getHost() const {
+        return host;
+    }
+
+    void setHost(const std::string &host) {
+        this->host = host;
+    }
+
+    const std::string &getPort() const {
+        return port;
+    }
+
+    void setPort(const std::string &port) {
+        this->port = port;
+    }
+
+    const std::string &getScheme() const {
+        return scheme;
+    }
+
+    void setScheme(const std::string &scheme) {
+        this->scheme = scheme;
+    }
+
+    const UserInfo &getUser() const {
+        return user;
+    }
+
+    void setUser(const UserInfo &user) {
+        this->user = user;
+    }
+
+    void addToken(const Token &token) {
+        user.addToken(token);
+    }
+
+private:
+    std::string authority;
+    std::string host;
+    std::string port;
+    std::string scheme;
+    UserInfo user;
+};
+}
+}
+
+HDFS_HASH_DEFINE(hdfs::internal::FileSystemKey);
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_FILESYSTEMKEY_H_ */

+ 77 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemStats.h

@@ -0,0 +1,77 @@
+/**
+ * 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_CLIENT_FSSTATS_H_
+#define _HDFS_LIBHDFS3_CLIENT_FSSTATS_H_
+
+#include <stdint.h>
+
+namespace hdfs {
+
+/**
+ * file system statistics
+ */
+class FileSystemStats {
+public:
+    /**
+     * To construct a FileSystemStats.
+     */
+    FileSystemStats() : capacity(-1), used(-1), remaining(-1) {
+    }
+
+    /**
+     * To construct a FileSystemStats with given values.
+     * @param capacity the capacity of file system.
+     * @param used the space which has been used.
+     * @param remaining available space on file system.
+     */
+    FileSystemStats(int64_t capacity, int64_t used, int64_t remaining)
+        : capacity(capacity), used(used), remaining(remaining) {
+    }
+
+    /**
+     * Return the capacity in bytes of the file system
+     * @return capacity of file system.
+     */
+    int64_t getCapacity() {
+        return capacity;
+    }
+
+    /**
+     * Return the number of bytes used on the file system
+     * @return return used space.
+     */
+    int64_t getUsed() {
+        return used;
+    }
+
+    /**
+     * Return the number of remaining bytes on the file system
+     * @return return available space.
+     */
+    int64_t getRemaining() {
+        return remaining;
+    }
+
+private:
+    int64_t capacity;
+    int64_t used;
+    int64_t remaining;
+};
+}
+#endif /* _HDFS_LIBHDFS3_CLIENT_FSSTATS_H_ */

+ 123 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc

@@ -0,0 +1,123 @@
+/**
+ * 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 "FileSystemImpl.h"
+#include "InputStream.h"
+#include "InputStreamImpl.h"
+#include "StatusInternal.h"
+
+using namespace hdfs::internal;
+
+namespace hdfs {
+
+InputStream::InputStream() {
+    impl = new internal::InputStreamImpl;
+}
+
+InputStream::~InputStream() {
+    delete impl;
+}
+
+Status InputStream::open(FileSystem &fs, const std::string &path,
+                         bool verifyChecksum) {
+    if (!fs.impl) {
+        THROW(HdfsIOException, "FileSystem: not connected.");
+    }
+
+    try {
+        impl->open(fs.impl, path.c_str(), verifyChecksum);
+    } catch (...) {
+        return lastError = CreateStatusFromException(current_exception());
+    }
+
+    return lastError = Status::OK();
+}
+
+int32_t InputStream::read(char *buf, int32_t size) {
+    int32_t retval = -1;
+
+    try {
+        retval = impl->read(buf, size);
+        lastError = Status::OK();
+    } catch (...) {
+        lastError = CreateStatusFromException(current_exception());
+    }
+
+    return retval;
+}
+
+Status InputStream::readFully(char *buf, int64_t size) {
+    try {
+        impl->readFully(buf, size);
+    } catch (...) {
+        return lastError = CreateStatusFromException(current_exception());
+    }
+
+    return lastError = Status::OK();
+}
+
+int64_t InputStream::available() {
+    int64_t retval = -1;
+
+    try {
+        retval = impl->available();
+        lastError = Status::OK();
+    } catch (...) {
+        lastError = CreateStatusFromException(current_exception());
+    }
+
+    return retval;
+}
+
+Status InputStream::seek(int64_t pos) {
+    try {
+        impl->seek(pos);
+    } catch (...) {
+        return lastError = CreateStatusFromException(current_exception());
+    }
+
+    return lastError = Status::OK();
+}
+
+int64_t InputStream::tell() {
+    int64_t retval = -1;
+
+    try {
+        retval = impl->tell();
+        lastError = Status::OK();
+    } catch (...) {
+        lastError = CreateStatusFromException(current_exception());
+    }
+
+    return retval;
+}
+
+Status InputStream::close() {
+    try {
+        impl->close();
+    } catch (...) {
+        return lastError = CreateStatusFromException(current_exception());
+    }
+
+    return lastError = Status::OK();
+}
+
+Status InputStream::getLastError() {
+    return lastError;
+}
+}

+ 35 - 12
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h

@@ -19,20 +19,28 @@
 #ifndef _HDFS_LIBHDFS3_CLIENT_INPUTSTREAM_H_
 #define _HDFS_LIBHDFS3_CLIENT_INPUTSTREAM_H_
 
-#include "FileSystem.h"
+#include "Status.h"
 
 namespace hdfs {
 namespace internal {
-class InputStreamInter;
+class InputStreamImpl;
 }
 
+class FileSystem;
+
 /**
  * A input stream used read data from hdfs.
  */
 class InputStream {
 public:
+    /**
+     * Construct an instance.
+     */
     InputStream();
 
+    /**
+     * Destroy this instance.
+     */
     ~InputStream();
 
     /**
@@ -40,51 +48,66 @@ public:
      * @param fs hdfs file system.
      * @param path the file to be read.
      * @param verifyChecksum verify the checksum.
+     * @return the result status of this operation
      */
-    void open(FileSystem & fs, const char * path, bool verifyChecksum = true);
+    Status open(FileSystem &fs, const std::string &path,
+                bool verifyChecksum = true);
 
     /**
      * To read data from hdfs.
      * @param buf the buffer used to filled.
      * @param size buffer size.
-     * @return return the number of bytes filled in the buffer, it may less than size.
+     * @return return the number of bytes filled in the buffer, it may less than
+     * size, -1 on error.
      */
-    int32_t read(char * buf, int32_t size);
+    int32_t read(char *buf, int32_t size);
 
     /**
      * To read data from hdfs, block until get the given size of bytes.
      * @param buf the buffer used to filled.
      * @param size the number of bytes to be read.
+     * @return the result status of this operation
      */
-    void readFully(char * buf, int64_t size);
+    Status readFully(char *buf, int64_t size);
 
     /**
      * Get how many bytes can be read without blocking.
-     * @return The number of bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking, -1 on error.
      */
     int64_t available();
 
     /**
      * To move the file point to the given position.
      * @param pos the given position.
+     * @return the result status of this operation
      */
-    void seek(int64_t pos);
+    Status seek(int64_t pos);
 
     /**
      * To get the current file point position.
-     * @return the position of current file point.
+     * @return the position of current file pointer, -1 on error.
      */
     int64_t tell();
 
     /**
      * Close the stream.
+     * @return the result status of this operation
      */
-    void close();
+    Status close();
+
+    /**
+     * Get the error status of the last operation.
+     * @return the error status of the last operation.
+     */
+    Status getLastError();
 
 private:
-    Internal::InputStreamInter * impl;
-};
+    InputStream(const InputStream &other);
+    InputStream &operator=(InputStream &other);
 
+    internal::InputStreamImpl *impl;
+    Status lastError;
+};
 }
 
 #endif /* _HDFS_LIBHDFS3_CLIENT_INPUTSTREAM_H_ */

+ 919 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.cc

@@ -0,0 +1,919 @@
+/**
+ * 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 "FileSystemImpl.h"
+#include "InputStreamImpl.h"
+#include "LocalBlockReader.h"
+#include "Logger.h"
+#include "RemoteBlockReader.h"
+#include "Thread.h"
+#include "UnorderedMap.h"
+#include "server/Datanode.h"
+
+#include <algorithm>
+#include <ifaddrs.h>
+#include <inttypes.h>
+#include <iostream>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+namespace hdfs {
+namespace internal {
+
+mutex InputStreamImpl::MutLocalBlockInforCache;
+unordered_map<uint32_t, shared_ptr<LocalBlockInforCacheType>>
+    InputStreamImpl::LocalBlockInforCache;
+
+unordered_set<std::string> BuildLocalAddrSet() {
+    unordered_set<std::string> set;
+    struct ifaddrs *ifAddr = NULL;
+    struct ifaddrs *pifAddr = NULL;
+    struct sockaddr *addr;
+
+    if (getifaddrs(&ifAddr)) {
+        THROW(HdfsNetworkException,
+              "InputStreamImpl: cannot get local network interface: %s",
+              GetSystemErrorInfo(errno));
+    }
+
+    try {
+        std::vector<char> host;
+        const char *pHost;
+        host.resize(INET6_ADDRSTRLEN + 1);
+
+        for (pifAddr = ifAddr; pifAddr != NULL; pifAddr = pifAddr->ifa_next) {
+            addr = pifAddr->ifa_addr;
+            memset(&host[0], 0, INET6_ADDRSTRLEN + 1);
+
+            if (addr->sa_family == AF_INET) {
+                pHost = inet_ntop(
+                    addr->sa_family,
+                    &(reinterpret_cast<struct sockaddr_in *>(addr))->sin_addr,
+                    &host[0], INET6_ADDRSTRLEN);
+            } else if (addr->sa_family == AF_INET6) {
+                pHost = inet_ntop(
+                    addr->sa_family,
+                    &(reinterpret_cast<struct sockaddr_in6 *>(addr))->sin6_addr,
+                    &host[0], INET6_ADDRSTRLEN);
+            } else {
+                continue;
+            }
+
+            if (NULL == pHost) {
+                THROW(HdfsNetworkException,
+                      "InputStreamImpl: cannot get convert network address "
+                      "to textual form: %s",
+                      GetSystemErrorInfo(errno));
+            }
+
+            set.insert(pHost);
+        }
+
+        /*
+         * add hostname.
+         */
+        long hostlen = sysconf(_SC_HOST_NAME_MAX);
+        host.resize(hostlen + 1);
+
+        if (gethostname(&host[0], host.size())) {
+            THROW(HdfsNetworkException,
+                  "InputStreamImpl: cannot get hostname: %s",
+                  GetSystemErrorInfo(errno));
+        }
+
+        set.insert(&host[0]);
+    } catch (...) {
+        if (ifAddr != NULL) {
+            freeifaddrs(ifAddr);
+        }
+
+        throw;
+    }
+
+    if (ifAddr != NULL) {
+        freeifaddrs(ifAddr);
+    }
+
+    return set;
+}
+
+InputStreamImpl::InputStreamImpl()
+    : closed(true),
+      localRead(true),
+      readFromUnderConstructedBlock(false),
+      verify(true),
+      maxGetBlockInfoRetry(3),
+      cursor(0),
+      endOfCurBlock(0),
+      lastBlockBeingWrittenLength(0),
+      prefetchSize(0) {
+#ifdef MOCK
+    stub = NULL;
+#endif
+}
+
+InputStreamImpl::~InputStreamImpl() {
+}
+
+void InputStreamImpl::checkStatus() {
+    if (closed) {
+        THROW(HdfsIOException, "InputStreamImpl: stream is not opened.");
+    }
+
+    if (lastError != exception_ptr()) {
+        rethrow_exception(lastError);
+    }
+}
+
+int64_t InputStreamImpl::readBlockLength(const LocatedBlock &b) {
+    const std::vector<DatanodeInfo> &nodes = b.getLocations();
+    int replicaNotFoundCount = nodes.size();
+
+    for (size_t i = 0; i < nodes.size(); ++i) {
+        try {
+            int64_t n = 0;
+            shared_ptr<Datanode> dn;
+            RpcAuth a = auth;
+            a.getUser().addToken(b.getToken());
+#ifdef MOCK
+
+            if (stub) {
+                dn = stub->getDatanode();
+            } else {
+                dn = shared_ptr<Datanode>(
+                    new DatanodeImpl(nodes[i].getIpAddr().c_str(),
+                                     nodes[i].getIpcPort(), *conf, a));
+            }
+
+#else
+            dn = shared_ptr<Datanode>(new DatanodeImpl(
+                nodes[i].getIpAddr().c_str(), nodes[i].getIpcPort(), *conf, a));
+#endif
+            n = dn->getReplicaVisibleLength(b);
+
+            if (n >= 0) {
+                return n;
+            }
+        } catch (const ReplicaNotFoundException &e) {
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to get block "
+                "visible length for Block: %s file %s from Datanode: %s\n%s",
+                b.toString().c_str(), path.c_str(),
+                nodes[i].formatAddress().c_str(), GetExceptionDetail(e));
+            LOG(INFO,
+                "InputStreamImpl: retry get block visible length for Block: "
+                "%s file %s from other datanode",
+                b.toString().c_str(), path.c_str());
+            --replicaNotFoundCount;
+        } catch (const HdfsIOException &e) {
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to get block visible length for "
+                "Block: %s file %s from Datanode: %s\n%s",
+                b.toString().c_str(), path.c_str(),
+                nodes[i].formatAddress().c_str(), GetExceptionDetail(e));
+            LOG(INFO,
+                "InputStreamImpl: retry get block visible length for Block: "
+                "%s file %s from other datanode",
+                b.toString().c_str(), path.c_str());
+        }
+    }
+
+    // Namenode told us about these locations, but none know about the replica
+    // means that we hit the race between pipeline creation start and end.
+    // we require all 3 because some other exception could have happened
+    // on a DN that has it.  we want to report that error
+    if (replicaNotFoundCount == 0) {
+        return 0;
+    }
+
+    return -1;
+}
+
+/**
+ * Getting blocks locations'information from namenode
+ */
+void InputStreamImpl::updateBlockInfos() {
+    int retry = maxGetBlockInfoRetry;
+
+    for (int i = 0; i < retry; ++i) {
+        try {
+            if (!lbs) {
+                lbs = shared_ptr<LocatedBlocks>(new LocatedBlocks);
+            }
+
+            filesystem->getBlockLocations(path, cursor, prefetchSize, *lbs);
+
+            if (lbs->isLastBlockComplete()) {
+                lastBlockBeingWrittenLength = 0;
+            } else {
+                shared_ptr<LocatedBlock> last = lbs->getLastBlock();
+
+                if (!last) {
+                    lastBlockBeingWrittenLength = 0;
+                } else {
+                    lastBlockBeingWrittenLength = readBlockLength(*last);
+
+                    if (lastBlockBeingWrittenLength == -1) {
+                        if (i + 1 >= retry) {
+                            THROW(HdfsIOException,
+                                  "InputStreamImpl: failed "
+                                  "to get block visible length for Block: "
+                                  "%s from all Datanode.",
+                                  last->toString().c_str());
+                        } else {
+                            LOG(LOG_ERROR,
+                                "InputStreamImpl: failed to get block visible "
+                                "length for Block: %s file %s from all "
+                                "Datanode.",
+                                last->toString().c_str(), path.c_str());
+
+                            try {
+                                sleep_for(milliseconds(4000));
+                            } catch (...) {
+                            }
+
+                            continue;
+                        }
+                    }
+
+                    last->setNumBytes(lastBlockBeingWrittenLength);
+                }
+            }
+
+            return;
+        } catch (const HdfsRpcException &e) {
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to get block information "
+                "for file %s, %s",
+                path.c_str(), GetExceptionDetail(e));
+
+            if (i + 1 >= retry) {
+                throw;
+            }
+        }
+
+        LOG(INFO,
+            "InputStreamImpl: retry to get block information for "
+            "file: %s, already tried %d time(s).",
+            path.c_str(), i + 1);
+    }
+}
+
+int64_t InputStreamImpl::getFileLength() {
+    int64_t length = lbs->getFileLength();
+
+    if (!lbs->isLastBlockComplete()) {
+        length += lastBlockBeingWrittenLength;
+    }
+
+    return length;
+}
+
+void InputStreamImpl::seekToBlock(const LocatedBlock &lb) {
+    if (cursor >= lbs->getFileLength()) {
+        assert(!lbs->isLastBlockComplete());
+        readFromUnderConstructedBlock = true;
+    } else {
+        readFromUnderConstructedBlock = false;
+    }
+
+    assert(cursor >= lb.getOffset() &&
+           cursor < lb.getOffset() + lb.getNumBytes());
+    curBlock = shared_ptr<LocatedBlock>(new LocatedBlock(lb));
+    int64_t blockSize = curBlock->getNumBytes();
+    assert(blockSize > 0);
+    endOfCurBlock = blockSize + curBlock->getOffset();
+    failedNodes.clear();
+    blockReader.reset();
+}
+
+bool InputStreamImpl::choseBestNode() {
+    const std::vector<DatanodeInfo> &nodes = curBlock->getLocations();
+
+    for (size_t i = 0; i < nodes.size(); ++i) {
+        if (std::binary_search(failedNodes.begin(), failedNodes.end(),
+                               nodes[i])) {
+            continue;
+        }
+
+        curNode = nodes[i];
+        return true;
+    }
+
+    return false;
+}
+
+bool InputStreamImpl::isLocalNode() {
+    static const unordered_set<std::string> LocalAddrSet = BuildLocalAddrSet();
+    bool retval = LocalAddrSet.find(curNode.getIpAddr()) != LocalAddrSet.end();
+    return retval;
+}
+
+BlockLocalPathInfo InputStreamImpl::getBlockLocalPathInfo(
+    LocalBlockInforCacheType &cache, const LocatedBlock &b) {
+    BlockLocalPathInfo retval;
+
+    try {
+        if (!cache.find(LocalBlockInforCacheKey(b.getBlockId(), b.getPoolId()),
+                        retval)) {
+            RpcAuth a = auth;
+            /*
+             * only kerberos based authentication is allowed, do not add token
+             */
+            shared_ptr<Datanode> dn = shared_ptr<Datanode>(new DatanodeImpl(
+                curNode.getIpAddr().c_str(), curNode.getIpcPort(), *conf, a));
+            dn->getBlockLocalPathInfo(b, b.getToken(), retval);
+            cache.insert(LocalBlockInforCacheKey(b.getBlockId(), b.getPoolId()),
+                         retval);
+        }
+    } catch (const HdfsIOException &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(
+            HdfsIOException,
+            "InputStreamImpl: Failed to get block local path information.");
+    }
+
+    return retval;
+}
+
+void InputStreamImpl::invalidCacheEntry(LocalBlockInforCacheType &cache,
+                                        const LocatedBlock &b) {
+    cache.erase(LocalBlockInforCacheKey(b.getBlockId(), b.getPoolId()));
+}
+
+LocalBlockInforCacheType &InputStreamImpl::getBlockLocalPathInfoCache(
+    uint32_t port) {
+    lock_guard<mutex> lock(MutLocalBlockInforCache);
+    unordered_map<uint32_t, shared_ptr<LocalBlockInforCacheType>>::iterator it;
+    it = LocalBlockInforCache.find(port);
+
+    if (it == LocalBlockInforCache.end()) {
+        shared_ptr<LocalBlockInforCacheType> retval;
+        retval =
+            shared_ptr<LocalBlockInforCacheType>(new LocalBlockInforCacheType(
+                conf->getMaxLocalBlockInfoCacheSize()));
+        LocalBlockInforCache[port] = retval;
+        return *retval;
+    } else {
+        return *(it->second);
+    }
+}
+
+void InputStreamImpl::setupBlockReader(bool temporaryDisableLocalRead) {
+    bool lastReadFromLocal = false;
+    exception_ptr lastException;
+
+    while (true) {
+        if (!choseBestNode()) {
+            try {
+                if (lastException) {
+                    rethrow_exception(lastException);
+                }
+            } catch (...) {
+                NESTED_THROW(
+                    HdfsIOException,
+                    "InputStreamImpl: all nodes have been tried and no valid "
+                    "replica can be read for Block: %s.",
+                    curBlock->toString().c_str());
+            }
+
+            THROW(HdfsIOException,
+                  "InputStreamImpl: all nodes have been tried and no valid "
+                  "replica can be read for Block: %s.",
+                  curBlock->toString().c_str());
+        }
+
+        try {
+            int64_t offset, len;
+            offset = cursor - curBlock->getOffset();
+            assert(offset >= 0);
+            len = curBlock->getNumBytes() - offset;
+            assert(len > 0);
+
+            if (auth.getProtocol() == AuthProtocol::NONE &&
+                !temporaryDisableLocalRead && !lastReadFromLocal &&
+                !readFromUnderConstructedBlock && localRead && isLocalNode()) {
+                lastReadFromLocal = true;
+                LocalBlockInforCacheType &cache =
+                    getBlockLocalPathInfoCache(curNode.getXferPort());
+                BlockLocalPathInfo info =
+                    getBlockLocalPathInfo(cache, *curBlock);
+                assert(curBlock->getBlockId() == info.getBlock().getBlockId() &&
+                       curBlock->getPoolId() == info.getBlock().getPoolId());
+                LOG(DEBUG2,
+                    "%p setup local block reader for file %s from "
+                    "local block %s, block offset %" PRId64
+                    ", read block "
+                    "length %" PRId64 " end of Block %" PRId64
+                    ", local "
+                    "block file path %s",
+                    this, path.c_str(), curBlock->toString().c_str(), offset,
+                    len, offset + len, info.getLocalBlockPath());
+
+                if (0 != access(info.getLocalMetaPath(), R_OK)) {
+                    invalidCacheEntry(cache, *curBlock);
+                    continue;
+                }
+
+                try {
+                    blockReader = shared_ptr<BlockReader>(
+                        new LocalBlockReader(info, *curBlock, offset, verify,
+                                             *conf, localReaderBuffer));
+                } catch (...) {
+                    invalidCacheEntry(cache, *curBlock);
+                    throw;
+                }
+            } else {
+                const char *clientName;
+                LOG(DEBUG2,
+                    "%p setup remote block reader for file %s from "
+                    "remote block %s, block offset %" PRId64
+                    ""
+                    ", read block length %" PRId64 " end of block %" PRId64
+                    ", from node %s",
+                    this, path.c_str(), curBlock->toString().c_str(), offset,
+                    len, offset + len, curNode.formatAddress().c_str());
+                clientName = filesystem->getClientName();
+                lastReadFromLocal = false;
+                blockReader = shared_ptr<BlockReader>(new RemoteBlockReader(
+                    *curBlock, curNode, offset, len, curBlock->getToken(),
+                    clientName, verify, *conf));
+            }
+
+            break;
+        } catch (const HdfsIOException &e) {
+            lastException = current_exception();
+
+            if (lastReadFromLocal) {
+                LOG(LOG_ERROR,
+                    "cannot setup block reader for Block: %s file %s "
+                    "on Datanode: %s.\n%s\n"
+                    "retry the same node but disable reading from local block",
+                    curBlock->toString().c_str(), path.c_str(),
+                    curNode.formatAddress().c_str(), GetExceptionDetail(e));
+                /*
+                 * do not add node into failedNodes since we will retry the same
+                 * node
+                 * but
+                 * disable local block reading
+                 */
+            } else {
+                LOG(LOG_ERROR,
+                    "cannot setup block reader for Block: %s file %s on "
+                    "Datanode: %s.\n%s\nretry another node",
+                    curBlock->toString().c_str(), path.c_str(),
+                    curNode.formatAddress().c_str(), GetExceptionDetail(e));
+                failedNodes.push_back(curNode);
+                std::sort(failedNodes.begin(), failedNodes.end());
+            }
+        }
+    }
+}
+
+void InputStreamImpl::open(shared_ptr<FileSystemImpl> fs, const char *path,
+                           bool verifyChecksum) {
+    if (NULL == path || 0 == strlen(path)) {
+        THROW(InvalidParameter, "path is invalid.");
+    }
+
+    try {
+        openInternal(fs, path, verifyChecksum);
+    } catch (...) {
+        close();
+        throw;
+    }
+}
+
+void InputStreamImpl::openInternal(shared_ptr<FileSystemImpl> fs,
+                                   const char *path, bool verifyChecksum) {
+    try {
+        filesystem = fs;
+        verify = verifyChecksum;
+        this->path = fs->getStandardPath(path);
+        LOG(DEBUG2, "%p, open file %s for read, verfyChecksum is %s", this,
+            this->path.c_str(), (verifyChecksum ? "true" : "false"));
+        conf = shared_ptr<SessionConfig>(new SessionConfig(fs->getConf()));
+        this->auth = RpcAuth(fs->getUserInfo(),
+                             RpcAuth::ParseMethod(conf->getRpcAuthMethod()));
+        prefetchSize = conf->getDefaultBlockSize() * conf->getPrefetchSize();
+        localRead = conf->isReadFromLocal();
+        maxGetBlockInfoRetry = conf->getMaxGetBlockInfoRetry();
+        updateBlockInfos();
+        closed = false;
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const FileNotFoundException &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException, "InputStreamImpl: cannot open file: %s.",
+                     this->path.c_str());
+    }
+}
+
+int32_t InputStreamImpl::read(char *buf, int32_t size) {
+    checkStatus();
+
+    try {
+        int64_t prvious = cursor;
+        int32_t done = readInternal(buf, size);
+        LOG(DEBUG3, "%p read file %s size is %d, offset %" PRId64
+                    " done %d, "
+                    "next pos %" PRId64,
+            this, path.c_str(), size, prvious, done, cursor);
+        return done;
+    } catch (const HdfsEndOfStream &e) {
+        throw;
+    } catch (...) {
+        lastError = current_exception();
+        throw;
+    }
+}
+
+int32_t InputStreamImpl::readOneBlock(char *buf, int32_t size,
+                                      bool shouldUpdateMetadataOnFailure) {
+    bool temporaryDisableLocalRead = false;
+
+    while (true) {
+        try {
+            /*
+             * Setup block reader here and handle failure.
+             */
+            if (!blockReader) {
+                setupBlockReader(temporaryDisableLocalRead);
+                temporaryDisableLocalRead = false;
+            }
+        } catch (const HdfsInvalidBlockToken &e) {
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to read Block: %s file %s, \n%s, "
+                "retry after updating block informations.",
+                curBlock->toString().c_str(), path.c_str(),
+                GetExceptionDetail(e));
+            return -1;
+        } catch (const HdfsIOException &e) {
+            /*
+             * In setupBlockReader, we have tried all the replicas.
+             * We now update block informations once, and try again.
+             */
+            if (shouldUpdateMetadataOnFailure) {
+                LOG(LOG_ERROR,
+                    "InputStreamImpl: failed to read Block: %s file %s, \n%s, "
+                    "retry after updating block informations.",
+                    curBlock->toString().c_str(), path.c_str(),
+                    GetExceptionDetail(e));
+                return -1;
+            } else {
+                /*
+                 * We have updated block informations and failed again.
+                 */
+                throw;
+            }
+        }
+
+        /*
+         * Block reader has been setup, read from block reader.
+         */
+        try {
+            int32_t todo = size;
+            todo = todo < endOfCurBlock - cursor
+                       ? todo
+                       : static_cast<int32_t>(endOfCurBlock - cursor);
+            assert(blockReader);
+            todo = blockReader->read(buf, todo);
+            cursor += todo;
+            /*
+             * Exit the loop and function from here if success.
+             */
+            return todo;
+        } catch (const HdfsIOException &e) {
+            /*
+             * Failed to read from current block reader,
+             * add the current datanode to invalid node list and try again.
+             */
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to read Block: %s file %s from "
+                "Datanode: %s, \n%s, "
+                "retry read again from another Datanode.",
+                curBlock->toString().c_str(), path.c_str(),
+                curNode.formatAddress().c_str(), GetExceptionDetail(e));
+
+            if (conf->doesNotRetryAnotherNode()) {
+                throw;
+            }
+        } catch (const ChecksumException &e) {
+            LOG(LOG_ERROR,
+                "InputStreamImpl: failed to read Block: %s file %s "
+                "from Datanode: %s, \n%s, retry read again from "
+                "another Datanode.",
+                curBlock->toString().c_str(), path.c_str(),
+                curNode.formatAddress().c_str(), GetExceptionDetail(e));
+        }
+
+        /*
+         * Successfully create the block reader but failed to read.
+         * Disable the local block reader and try the same node again.
+         */
+        if (!blockReader ||
+            dynamic_cast<LocalBlockReader *>(blockReader.get())) {
+            temporaryDisableLocalRead = true;
+        } else {
+            /*
+             * Remote block reader failed to read, try another node.
+             */
+            LOG(INFO,
+                "IntputStreamImpl: Add invalid datanode %s to failed "
+                "datanodes and try another datanode again for file %s.",
+                curNode.formatAddress().c_str(), path.c_str());
+            failedNodes.push_back(curNode);
+            std::sort(failedNodes.begin(), failedNodes.end());
+        }
+
+        blockReader.reset();
+    }
+}
+
+/**
+ * To read data from hdfs.
+ * @param buf the buffer used to filled.
+ * @param size buffer size.
+ * @return return the number of bytes filled in the buffer, it may less than
+ * size.
+ */
+int32_t InputStreamImpl::readInternal(char *buf, int32_t size) {
+    int updateMetadataOnFailure = conf->getMaxReadBlockRetry();
+
+    try {
+        do {
+            const LocatedBlock *lb = NULL;
+
+            /*
+             * Check if we have got the block information we need.
+             */
+            if (!lbs || cursor >= getFileLength() ||
+                (cursor >= endOfCurBlock && !(lb = lbs->findBlock(cursor)))) {
+                /*
+                 * Get block information from namenode.
+                 * Do RPC failover work in updateBlockInfos.
+                 */
+                updateBlockInfos();
+
+                /*
+                 * We already have the up-to-date block information,
+                 * Check if we reach the end of file.
+                 */
+                if (cursor >= getFileLength()) {
+                    THROW(HdfsEndOfStream,
+                          "InputStreamImpl: read over EOF, current position: "
+                          "%" PRId64 ", read size: %d, from file: %s",
+                          cursor, size, path.c_str());
+                }
+            }
+
+            /*
+             * If we reach the end of block or the block information has just
+             * updated,
+             * seek to the right block to read.
+             */
+            if (cursor >= endOfCurBlock) {
+                lb = lbs->findBlock(cursor);
+
+                if (!lb) {
+                    THROW(HdfsIOException,
+                          "InputStreamImpl: cannot find block information at "
+                          "position: %" PRId64 " for file: %s",
+                          cursor, path.c_str());
+                }
+
+                /*
+                 * Seek to the right block, setup all needed variable,
+                 * but do not setup block reader, setup it latter.
+                 */
+                seekToBlock(*lb);
+            }
+
+            int32_t retval =
+                readOneBlock(buf, size, updateMetadataOnFailure > 0);
+
+            /*
+             * Now we have tried all replicas and failed.
+             * We will update metadata once and try again.
+             */
+            if (retval < 0) {
+                lbs.reset();
+                endOfCurBlock = 0;
+                --updateMetadataOnFailure;
+
+                try {
+                    sleep_for(seconds(1));
+                } catch (...) {
+                }
+
+                continue;
+            }
+
+            return retval;
+        } while (true);
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsEndOfStream &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        /*
+         * wrap the underlying error and rethrow.
+         */
+        NESTED_THROW(HdfsIOException,
+                     "InputStreamImpl: cannot read file: %s, from "
+                     "position %" PRId64 ", size: %d.",
+                     path.c_str(), cursor, size);
+    }
+}
+
+/**
+ * To read data from hdfs, block until get the given size of bytes.
+ * @param buf the buffer used to filled.
+ * @param size the number of bytes to be read.
+ */
+void InputStreamImpl::readFully(char *buf, int64_t size) {
+    LOG(DEBUG3, "readFully file %s size is %" PRId64 ", offset %" PRId64,
+        path.c_str(), size, cursor);
+    checkStatus();
+
+    try {
+        return readFullyInternal(buf, size);
+    } catch (const HdfsEndOfStream &e) {
+        throw;
+    } catch (...) {
+        lastError = current_exception();
+        throw;
+    }
+}
+
+void InputStreamImpl::readFullyInternal(char *buf, int64_t size) {
+    int32_t done;
+    int64_t pos = cursor, todo = size;
+
+    try {
+        while (todo > 0) {
+            done = todo < std::numeric_limits<int32_t>::max()
+                       ? static_cast<int32_t>(todo)
+                       : std::numeric_limits<int32_t>::max();
+            done = readInternal(buf + (size - todo), done);
+            todo -= done;
+        }
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsEndOfStream &e) {
+        THROW(HdfsEndOfStream,
+              "InputStreamImpl: read over EOF, current position: %" PRId64
+              ", read size: %" PRId64 ", from file: %s",
+              pos, size, path.c_str());
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "InputStreamImpl: cannot read fully from file: %s, "
+                     "from position %" PRId64 ", size: %" PRId64 ".",
+                     path.c_str(), pos, size);
+    }
+}
+
+int64_t InputStreamImpl::available() {
+    checkStatus();
+
+    try {
+        if (blockReader) {
+            return blockReader->available();
+        }
+    } catch (...) {
+        lastError = current_exception();
+        throw;
+    }
+
+    return 0;
+}
+
+/**
+ * To move the file point to the given position.
+ * @param size the given position.
+ */
+void InputStreamImpl::seek(int64_t pos) {
+    LOG(DEBUG2, "%p seek file %s to %" PRId64 ", offset %" PRId64, this,
+        path.c_str(), pos, cursor);
+    checkStatus();
+
+    try {
+        seekInternal(pos);
+    } catch (...) {
+        lastError = current_exception();
+        throw;
+    }
+}
+
+void InputStreamImpl::seekInternal(int64_t pos) {
+    if (cursor == pos) {
+        return;
+    }
+
+    if (!lbs || pos > getFileLength()) {
+        updateBlockInfos();
+
+        if (pos > getFileLength()) {
+            THROW(HdfsEndOfStream,
+                  "InputStreamImpl: seek over EOF, current position: %" PRId64
+                  ", seek target: %" PRId64 ", in file: %s",
+                  cursor, pos, path.c_str());
+        }
+    }
+
+    try {
+        if (blockReader && pos > cursor && pos < endOfCurBlock) {
+            blockReader->skip(pos - cursor);
+            cursor = pos;
+            return;
+        }
+    } catch (const HdfsIOException &e) {
+        LOG(LOG_ERROR, "InputStreamImpl: failed to skip %" PRId64
+                       " bytes in current block reader for file %s\n%s",
+            pos - cursor, path.c_str(), GetExceptionDetail(e));
+        LOG(INFO, "InputStreamImpl: retry to seek to position %" PRId64
+                  " for file %s",
+            pos, path.c_str());
+    } catch (const ChecksumException &e) {
+        LOG(LOG_ERROR, "InputStreamImpl: failed to skip %" PRId64
+                       " bytes in current block reader for file %s\n%s",
+            pos - cursor, path.c_str(), GetExceptionDetail(e));
+        LOG(INFO, "InputStreamImpl: retry to seek to position %" PRId64
+                  " for file %s",
+            pos, path.c_str());
+    }
+
+    /**
+     * the seek target exceed the current block or skip failed in current block
+     * reader.
+     * reset current block reader and set the cursor to the target position to
+     * seek.
+     */
+    endOfCurBlock = 0;
+    blockReader.reset();
+    cursor = pos;
+}
+
+/**
+ * To get the current file point position.
+ * @return the position of current file point.
+ */
+int64_t InputStreamImpl::tell() {
+    checkStatus();
+    LOG(DEBUG2, "tell file %s at %" PRId64, path.c_str(), cursor);
+    return cursor;
+}
+
+/**
+ * Close the stream.
+ */
+void InputStreamImpl::close() {
+    LOG(DEBUG2, "%p close file %s for read", this, path.c_str());
+    closed = true;
+    localRead = true;
+    readFromUnderConstructedBlock = false;
+    verify = true;
+    filesystem.reset();
+    cursor = 0;
+    endOfCurBlock = 0;
+    lastBlockBeingWrittenLength = 0;
+    prefetchSize = 0;
+    blockReader.reset();
+    curBlock.reset();
+    lbs.reset();
+    conf.reset();
+    failedNodes.clear();
+    path.clear();
+    localReaderBuffer.resize(0);
+    lastError = exception_ptr();
+}
+
+std::string InputStreamImpl::toString() {
+    if (path.empty()) {
+        return std::string("InputStream for path ") + path;
+    } else {
+        return std::string("InputStream (not opened)");
+    }
+}
+}
+}

+ 42 - 38
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h

@@ -19,13 +19,10 @@
 #ifndef _HDFS_LIBHDFS3_CLIENT_INPUTSTREAMIMPL_H_
 #define _HDFS_LIBHDFS3_CLIENT_INPUTSTREAMIMPL_H_
 
-#include "platform.h"
-
 #include "BlockReader.h"
 #include "ExceptionInternal.h"
 #include "FileSystem.h"
 #include "Hash.h"
-#include "InputStreamInter.h"
 #include "LruMap.h"
 #include "SessionConfig.h"
 #include "SharedPtr.h"
@@ -45,14 +42,16 @@ namespace hdfs {
 namespace internal {
 
 typedef std::pair<int64_t, std::string> LocalBlockInforCacheKey;
-typedef LruMap<LocalBlockInforCacheKey, BlockLocalPathInfo> LocalBlockInforCacheType;
+typedef LruMap<LocalBlockInforCacheKey, BlockLocalPathInfo>
+    LocalBlockInforCacheType;
 
 /**
  * A input stream used read data from hdfs.
  */
-class InputStreamImpl: public InputStreamInter {
+class InputStreamImpl {
 public:
     InputStreamImpl();
+
     ~InputStreamImpl();
 
     /**
@@ -61,22 +60,24 @@ public:
      * @param path the file to be read.
      * @param verifyChecksum verify the checksum.
      */
-    void open(shared_ptr<FileSystemInter> fs, const char * path, bool verifyChecksum);
+    void open(shared_ptr<FileSystemImpl> fs, const char *path,
+              bool verifyChecksum);
 
     /**
      * To read data from hdfs.
      * @param buf the buffer used to filled.
      * @param size buffer size.
-     * @return return the number of bytes filled in the buffer, it may less than size.
+     * @return return the number of bytes filled in the buffer, it may less than
+     * size.
      */
-    int32_t read(char * buf, int32_t size);
+    int32_t read(char *buf, int32_t size);
 
     /**
      * To read data from hdfs, block until get the given size of bytes.
      * @param buf the buffer used to filled.
      * @param size the number of bytes to be read.
      */
-    void readFully(char * buf, int64_t size);
+    void readFully(char *buf, int64_t size);
 
     int64_t available();
 
@@ -100,23 +101,24 @@ public:
     std::string toString();
 
 private:
-    BlockLocalPathInfo getBlockLocalPathInfo(LocalBlockInforCacheType & cache,
-            const LocatedBlock & b);
+    BlockLocalPathInfo getBlockLocalPathInfo(LocalBlockInforCacheType &cache,
+                                             const LocatedBlock &b);
     bool choseBestNode();
     bool isLocalNode();
-    int32_t readInternal(char * buf, int32_t size);
-    int32_t readOneBlock(char * buf, int32_t size, bool shouldUpdateMetadataOnFailure);
+    int32_t readInternal(char *buf, int32_t size);
+    int32_t readOneBlock(char *buf, int32_t size,
+                         bool shouldUpdateMetadataOnFailure);
     int64_t getFileLength();
-    int64_t readBlockLength(const LocatedBlock & b);
-    LocalBlockInforCacheType & getBlockLocalPathInfoCache(uint32_t port);
+    int64_t readBlockLength(const LocatedBlock &b);
+    LocalBlockInforCacheType &getBlockLocalPathInfoCache(uint32_t port);
     void checkStatus();
-    void invalidCacheEntry(LocalBlockInforCacheType & cache,
-                           const LocatedBlock & b);
-    void openInternal(shared_ptr<FileSystemInter> fs, const char * path,
+    void invalidCacheEntry(LocalBlockInforCacheType &cache,
+                           const LocatedBlock &b);
+    void openInternal(shared_ptr<FileSystemImpl> fs, const char *path,
                       bool verifyChecksum);
-    void readFullyInternal(char * buf, int64_t size);
+    void readFullyInternal(char *buf, int64_t size);
     void seekInternal(int64_t pos);
-    void seekToBlock(const LocatedBlock & lb);
+    void seekToBlock(const LocatedBlock &lb);
     void setupBlockReader(bool temporaryDisableLocalRead);
     void updateBlockInfos();
 
@@ -135,7 +137,7 @@ private:
     int64_t prefetchSize;
     RpcAuth auth;
     shared_ptr<BlockReader> blockReader;
-    shared_ptr<FileSystemInter> filesystem;
+    shared_ptr<FileSystemImpl> filesystem;
     shared_ptr<LocatedBlock> curBlock;
     shared_ptr<LocatedBlocks> lbs;
     shared_ptr<SessionConfig> conf;
@@ -144,28 +146,31 @@ private:
     std::vector<char> localReaderBuffer;
 
     static mutex MutLocalBlockInforCache;
-    static unordered_map<uint32_t, shared_ptr<LocalBlockInforCacheType> > LocalBlockInforCache;
-#ifdef MOCK
+    static unordered_map<uint32_t, shared_ptr<LocalBlockInforCacheType>>
+        LocalBlockInforCache;
+
 private:
-    hdfs::mock::TestDatanodeStub * stub;
+    InputStreamImpl(const InputStreamImpl &other);
+    InputStreamImpl &operator=(const InputStreamImpl &other);
+
+#ifdef MOCK
+    hdfs::mock::TestDatanodeStub *stub;
 #endif
 };
-
 }
 }
 
 #ifdef NEED_BOOST
 
 namespace boost {
-template<>
+template <>
 struct hash<hdfs::internal::LocalBlockInforCacheKey> {
     std::size_t operator()(
-        const hdfs::internal::LocalBlockInforCacheKey & key) const {
+        const hdfs::internal::LocalBlockInforCacheKey &key) const {
         size_t values[] = {hdfs::internal::Int64Hasher(key.first),
-                           hdfs::internal::StringHasher(key.second)
-                          };
-        return hdfs::internal::CombineHasher(values,
-                                             sizeof(values) / sizeof(values[0]));
+                           hdfs::internal::StringHasher(key.second)};
+        return hdfs::internal::CombineHasher(
+            values, sizeof(values) / sizeof(values[0]));
     }
 };
 }
@@ -173,15 +178,14 @@ struct hash<hdfs::internal::LocalBlockInforCacheKey> {
 #else
 
 namespace std {
-template<>
+template <>
 struct hash<hdfs::internal::LocalBlockInforCacheKey> {
     std::size_t operator()(
-        const hdfs::internal::LocalBlockInforCacheKey & key) const {
-        size_t values[] = { hdfs::internal::Int64Hasher(key.first),
-                            hdfs::internal::StringHasher(key.second)
-                          };
-        return hdfs::internal::CombineHasher(values,
-                                             sizeof(values) / sizeof(values[0]));
+        const hdfs::internal::LocalBlockInforCacheKey &key) const {
+        size_t values[] = {hdfs::internal::Int64Hasher(key.first),
+                           hdfs::internal::StringHasher(key.second)};
+        return hdfs::internal::CombineHasher(
+            values, sizeof(values) / sizeof(values[0]));
     }
 };
 }

+ 15 - 13
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h

@@ -32,11 +32,11 @@
 namespace hdfs {
 namespace internal {
 
-class LocalBlockReader: public BlockReader {
+class LocalBlockReader : public BlockReader {
 public:
-    LocalBlockReader(const BlockLocalPathInfo & info,
-                     const ExtendedBlock & block, int64_t offset, bool verify,
-                     SessionConfig & conf, std::vector<char> & buffer);
+    LocalBlockReader(const BlockLocalPathInfo &info, const ExtendedBlock &block,
+                     int64_t offset, bool verify, SessionConfig &conf,
+                     std::vector<char> &buffer);
 
     ~LocalBlockReader();
 
@@ -55,7 +55,7 @@ public:
      * @return return the number of bytes filled in the buffer,
      *  it may less than size. Return 0 if reach the end of block.
      */
-    virtual int32_t read(char * buf, int32_t size);
+    virtual int32_t read(char *buf, int32_t size);
 
     /**
      * Move the cursor forward len bytes.
@@ -64,34 +64,36 @@ public:
     virtual void skip(int64_t len);
 
 private:
+    LocalBlockReader(const LocalBlockReader &other);
+    LocalBlockReader &operator=(const LocalBlockReader &other);
+
     /**
      * Fill buffer and verify checksum.
      * @param bufferSize The size of buffer.
      */
     void readAndVerify(int32_t bufferSize);
-    int32_t readInternal(char * buf, int32_t len);
+    int32_t readInternal(char *buf, int32_t len);
 
 private:
-    bool verify; //verify checksum or not.
+    bool verify;  // verify checksum or not.
     const char *pbuffer;
     const char *pMetaBuffer;
     const ExtendedBlock &block;
     int checksumSize;
     int chunkSize;
     int localBufferSize;
-    int position; //point in buffer.
-    int size;  //data size in buffer.
-    int64_t cursor; //point in block.
-    int64_t length; //data size of block.
+    int position;    // point in buffer.
+    int size;        // data size in buffer.
+    int64_t cursor;  // point in block.
+    int64_t length;  // data size of block.
     shared_ptr<Checksum> checksum;
     shared_ptr<FileWrapper> dataFd;
     shared_ptr<FileWrapper> metaFd;
     std::string dataFilePath;
     std::string metaFilePath;
-    std::vector<char> & buffer;
+    std::vector<char> &buffer;
     std::vector<char> metaBuffer;
 };
-
 }
 }
 

+ 14 - 14
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h

@@ -29,11 +29,10 @@ namespace internal {
 
 class ConstPacketBuffer {
 public:
-    ConstPacketBuffer(const char * buf, int size) :
-        buffer(buf), size(size) {
+    ConstPacketBuffer(const char *buf, int size) : buffer(buf), size(size) {
     }
 
-    const char * getBuffer() const {
+    const char *getBuffer() const {
         return buffer;
     }
 
@@ -42,7 +41,7 @@ public:
     }
 
 private:
-    const char * buffer;
+    const char *buffer;
     const int size;
 };
 
@@ -65,13 +64,15 @@ public:
     /**
      * create a new packet
      */
-    Packet(int pktSize, int chunksPerPkt, int64_t offsetInBlock, int64_t seqno, int checksumSize);
+    Packet(int pktSize, int chunksPerPkt, int64_t offsetInBlock, int64_t seqno,
+           int checksumSize);
 
-    void reset(int pktSize, int chunksPerPkt, int64_t offsetInBlock, int64_t seqno, int checksumSize);
+    void reset(int pktSize, int chunksPerPkt, int64_t offsetInBlock,
+               int64_t seqno, int checksumSize);
 
     void addChecksum(uint32_t checksum);
 
-    void addData(const char * buf, int size);
+    void addData(const char *buf, int size);
 
     void setSyncFlag(bool sync);
 
@@ -102,21 +103,20 @@ public:
     }
 
 private:
-    bool lastPacketInBlock; // is this the last packet in block
-    bool syncBlock; // sync block to disk?
+    bool lastPacketInBlock;  // is this the last packet in block
+    bool syncBlock;          // sync block to disk?
     int checksumPos;
     int checksumSize;
     int checksumStart;
     int dataPos;
     int dataStart;
     int headerStart;
-    int maxChunks; // max chunks in packet
-    int numChunks; // number of chunks currently in packet
-    int64_t offsetInBlock; // offset in block
-    int64_t seqno; // sequence number of packet in block
+    int maxChunks;          // max chunks in packet
+    int numChunks;          // number of chunks currently in packet
+    int64_t offsetInBlock;  // offset in block
+    int64_t seqno;          // sequence number of packet in block
     std::vector<char> buffer;
 };
-
 }
 }
 #endif /* _HDFS_LIBHDFS3_CLIENT_PACKET_H_ */

+ 5 - 8
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h

@@ -28,6 +28,8 @@ namespace internal {
 
 class PacketHeader {
 public:
+    static int GetPkgHeaderSize();
+    static int CalcPkgHeaderSize();
     PacketHeader();
     PacketHeader(int packetLen, int64_t offsetInBlock, int64_t seqno,
                  bool lastPacketInBlock, int dataLen);
@@ -37,24 +39,19 @@ public:
     int getPacketLen();
     int64_t getOffsetInBlock();
     int64_t getSeqno();
-    void readFields(const char * buf, size_t size);
+    void readFields(const char *buf, size_t size);
+
     /**
      * Write the header into the buffer.
      * This requires that PKT_HEADER_LEN bytes are available.
      */
-    void writeInBuffer(char * buf, size_t size);
-
-public:
-    static int GetPkgHeaderSize();
-    static int CalcPkgHeaderSize();
+    void writeInBuffer(char *buf, size_t size);
 
 private:
     static int PkgHeaderSize;
-private:
     int32_t packetLen;
     PacketHeaderProto proto;
 };
-
 }
 }
 

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h

@@ -35,7 +35,7 @@
 namespace hdfs {
 namespace internal {
 
-class RemoteBlockReader: public BlockReader {
+class RemoteBlockReader : public BlockReader {
 public:
     RemoteBlockReader(const ExtendedBlock &eb, DatanodeInfo &datanode,
                       int64_t start, int64_t len, const Token &token,
@@ -65,6 +65,8 @@ public:
     virtual void skip(int64_t len);
 
 private:
+    RemoteBlockReader(const RemoteBlockReader &other);
+    RemoteBlockReader &operator=(RemoteBlockReader &other);
     bool readTrailingEmptyPacket();
     shared_ptr<PacketHeader> readPacketHeader();
     void checkResponse();
@@ -72,20 +74,19 @@ private:
     void sendStatus();
     void verifyChecksum(int chunks);
 
-private:
-    bool verify; //verify checksum or not.
+    bool verify;  // verify checksum or not.
     DatanodeInfo &datanode;
     const ExtendedBlock &binfo;
     int checksumSize;
     int chunkSize;
     int connTimeout;
-    int position; //point in buffer.
+    int position;  // point in buffer.
     int readTimeout;
-    int size;  //data size in buffer.
+    int size;  // data size in buffer.
     int writeTimeout;
-    int64_t cursor; //point in block.
-    int64_t endOffset; //offset in block requested to read to.
-    int64_t lastSeqNo; //segno of the last chunk received
+    int64_t cursor;     // point in block.
+    int64_t endOffset;  // offset in block requested to read to.
+    int64_t lastSeqNo;  // segno of the last chunk received
     shared_ptr<BufferedSocketReader> in;
     shared_ptr<Checksum> checksum;
     shared_ptr<DataTransferProtocol> sender;
@@ -93,7 +94,6 @@ private:
     shared_ptr<Socket> sock;
     std::vector<char> buffer;
 };
-
 }
 }
 

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/TokenInternal.h

@@ -0,0 +1,27 @@
+/**
+ * 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_CLIENT_TOKENINTERNAL_H_
+#define _HDFS_LIBHDFS3_CLIENT_TOKENINTERNAL_H_
+
+#include "Hash.h"
+#include "Token.h"
+
+HDFS_HASH_DEFINE(::hdfs::internal::Token);
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_TOKENINTERNAL_H_ */

+ 213 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.cc

@@ -0,0 +1,213 @@
+/**
+ * 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 "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);
+}
+
+Config::Config(ConfigImpl *impl) : impl(impl) {
+}
+
+Config &Config::operator=(const Config &other) {
+    if (this == &other) {
+        return *this;
+    }
+
+    ConfigImpl *temp = impl;
+    impl = new ConfigImpl(*other.impl);
+    delete temp;
+    return *this;
+}
+
+bool Config::operator==(const Config &other) const {
+    if (this == &other) {
+        return true;
+    }
+
+    return *impl == *other.impl;
+}
+
+Config::~Config() {
+    delete impl;
+}
+
+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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    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());
+    }
+
+    return Status::OK();
+}
+
+void Config::set(const std::string &key, const std::string &value) {
+    impl->set(key.c_str(), value);
+}
+
+void Config::set(const std::string &key, int32_t value) {
+    impl->set(key.c_str(), value);
+}
+
+void Config::set(const std::string &key, int64_t value) {
+    impl->set(key.c_str(), value);
+}
+
+void Config::set(const std::string &key, double value) {
+    impl->set(key.c_str(), value);
+}
+
+void Config::set(const std::string &key, bool value) {
+    impl->set(key.c_str(), value);
+}
+
+size_t Config::hash_value() const {
+    return impl->hash_value();
+}
+}

+ 218 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Config.h

@@ -0,0 +1,218 @@
+/**
+ * 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_CONFIG_H_
+#define _HDFS_LIBHDFS3_COMMON_CONFIG_H_
+
+#include "Status.h"
+
+#include <stdint.h>
+#include <string>
+#include <map>
+
+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();
+
+    /**
+     * Get a string with given configure key.
+     * @param key The key of the configure item.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getString(const std::string &key, std::string *output) 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.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getString(const std::string &key, const std::string &def,
+                     std::string *output) const;
+
+    /**
+     * Get a 64 bit integer with given configure key.
+     * @param key The key of the configure item.
+     * @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;
+
+    /**
+     * 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.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getInt64(const std::string &key, int64_t def,
+                    std::string *output) const;
+
+    /**
+     * Get a 32 bit integer with given configure key.
+     * @param key The key of the configure item.
+     * @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;
+
+    /**
+     * 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.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getInt32(const std::string &key, int32_t def,
+                    std::string *output) const;
+
+    /**
+     * Get a double with given configure key.
+     * @param key The key of the configure item.
+     * @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;
+
+    /**
+     * 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.
+     * @param output the pointer of the output parameter.
+     * @return the result status of this operation
+     */
+    Status getDouble(const std::string &key, double def,
+                     std::string *output) const;
+
+    /**
+     * Get a boolean with given configure key.
+     * @param key The key of the configure item.
+     * @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;
+
+    /**
+     * 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.
+     * @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;
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    void set(const std::string &key, const std::string &value);
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    void set(const std::string &key, int32_t value);
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    void set(const std::string &key, int64_t value);
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    void set(const std::string &key, double value);
+
+    /**
+     * Set a configure item
+     * @param key The key will set.
+     * @param value The value will be set to.
+     */
+    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;
+};
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_CONFIG_H_ */

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

@@ -0,0 +1,291 @@
+/**
+ * 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());
+}
+}
+}

+ 28 - 45
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfig.h → hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/ConfigImpl.h

@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef _HDFS_LIBHDFS3_COMMON_XMLCONFIG_H_
-#define _HDFS_LIBHDFS3_COMMON_XMLCONFIG_H_
+#ifndef _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_
+#define _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_
 
 #include <stdint.h>
 #include <string>
@@ -25,47 +25,31 @@
 #include <map>
 
 namespace hdfs {
+namespace internal {
 
 /**
  * A configure file parser.
  */
-class Config {
+class ConfigImpl {
 public:
-    /**
-     * Construct a empty Config instance.
-     */
-    Config() {
+    ConfigImpl() {
     }
 
     /**
-     * Construct a Config with given configure file.
-     * @param path The path of configure file.
-     * @throw HdfsBadConfigFoumat
-     */
-    Config(const char *path);
-
-    /**
-     * Parse the configure file.
-     * @throw HdfsBadConfigFoumat
+     * Construct a empty Config instance.
      */
-    void update(const char *path);
+    ConfigImpl(const std::map<std::string, std::string> &kv);
 
     /**
-     * Get a string with given configure key.
-     * @param key The key of the configure item.
-     * @return The value of configure item.
-     * @throw HdfsConfigNotFound
+     * Operator equal
      */
-    const char *getString(const char *key) const;
+    bool operator==(const ConfigImpl &other) const {
+        if (this == &other) {
+            return true;
+        }
 
-    /**
-     * 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 char *key, const char *def) const;
+        return this->kv == other.kv;
+    }
 
     /**
      * Get a string with given configure key.
@@ -73,7 +57,7 @@ public:
      * @return The value of configure item.
      * @throw HdfsConfigNotFound
      */
-    const char *getString(const std::string & key) const;
+    const char *getString(const std::string &key) const;
 
     /**
      * Get a string with given configure key.
@@ -82,8 +66,7 @@ public:
      * @param def The defalut value.
      * @return The value of configure item.
      */
-    const char *getString(const std::string & key,
-                           const std::string & def) const;
+    const char *getString(const std::string &key, const std::string &def) const;
 
     /**
      * Get a 64 bit integer with given configure key.
@@ -91,7 +74,7 @@ public:
      * @return The value of configure item.
      * @throw HdfsConfigNotFound
      */
-    int64_t getInt64(const char *key) const;
+    int64_t getInt64(const std::string &key) const;
 
     /**
      * Get a 64 bit integer with given configure key.
@@ -100,7 +83,7 @@ public:
      * @param def The defalut value.
      * @return The value of configure item.
      */
-    int64_t getInt64(const char *key, int64_t def) const;
+    int64_t getInt64(const std::string &key, int64_t def) const;
 
     /**
      * Get a 32 bit integer with given configure key.
@@ -108,7 +91,7 @@ public:
      * @return The value of configure item.
      * @throw HdfsConfigNotFound
      */
-    int32_t getInt32(const char *key) const;
+    int32_t getInt32(const std::string &key) const;
 
     /**
      * Get a 32 bit integer with given configure key.
@@ -117,7 +100,7 @@ public:
      * @param def The defalut value.
      * @return The value of configure item.
      */
-    int32_t getInt32(const char *key, int32_t def) const;
+    int32_t getInt32(const std::string &key, int32_t def) const;
 
     /**
      * Get a double with given configure key.
@@ -125,7 +108,7 @@ public:
      * @return The value of configure item.
      * @throw HdfsConfigNotFound
      */
-    double getDouble(const char *key) const;
+    double getDouble(const std::string &key) const;
 
     /**
      * Get a double with given configure key.
@@ -134,7 +117,7 @@ public:
      * @param def The defalut value.
      * @return The value of configure item.
      */
-    double getDouble(const char *key, double def) const;
+    double getDouble(const std::string &key, double def) const;
 
     /**
      * Get a boolean with given configure key.
@@ -142,7 +125,7 @@ public:
      * @return The value of configure item.
      * @throw HdfsConfigNotFound
      */
-    bool getBool(const char *key) const;
+    bool getBool(const std::string &key) const;
 
     /**
      * Get a boolean with given configure key.
@@ -151,15 +134,15 @@ public:
      * @param def The default value.
      * @return The value of configure item.
      */
-    bool getBool(const char *key, bool def) const;
+    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 char *key, T const & value) {
+    template <typename T>
+    void set(const std::string &key, T const &value) {
         std::stringstream ss;
         ss << value;
         kv[key] = ss.str();
@@ -176,7 +159,7 @@ private:
     std::string path;
     std::map<std::string, std::string> kv;
 };
-
+}
 }
 
-#endif /* _HDFS_LIBHDFS3_COMMON_XMLCONFIG_H_ */
+#endif /* _HDFS_LIBHDFS3_COMMON_XMLCONFIGIMPL_H_ */

+ 68 - 95
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.cc

@@ -29,8 +29,8 @@
 namespace hdfs {
 namespace internal {
 
-template<typename T>
-static void CheckRangeGE(const char *key, T const & value, T const & target) {
+template <typename T>
+static void CheckRangeGE(const char *key, T const &value, T const &target) {
     if (!(value >= target)) {
         std::stringstream ss;
         ss << "Invalid configure item: \"" << key << "\", value: " << value
@@ -39,99 +39,73 @@ static void CheckRangeGE(const char *key, T const & value, T const & target) {
     }
 }
 
-template<typename T>
-static void CheckMultipleOf(const char *key, const T & value, int unit) {
+template <typename T>
+static void CheckMultipleOf(const char *key, const T &value, int unit) {
     if (value <= 0 || value % unit != 0) {
-        THROW(HdfsConfigInvalid, "%s should be larger than 0 and be the multiple of %d.", key, unit);
+        THROW(HdfsConfigInvalid,
+              "%s should be larger than 0 and be the multiple of %d.", key,
+              unit);
     }
 }
 
-SessionConfig::SessionConfig(const Config & conf) {
-    ConfigDefault<bool> boolValues [] = {
-        {
-            &rpcTcpNoDelay, "rpc.client.connect.tcpnodelay", true
-        }, {
-            &readFromLocal, "dfs.client.read.shortcircuit", true
-        }, {
-            &addDatanode, "output.replace-datanode-on-failure", true
-        }, {
-            &notRetryAnotherNode, "input.notretry-another-node", false
-        }, {
-            &useMappedFile, "input.localread.mappedfile", true
-        }
-    };
+SessionConfig::SessionConfig(const ConfigImpl &conf) {
+    ConfigDefault<bool> boolValues[] = {
+        {&rpcTcpNoDelay, "rpc.client.connect.tcpnodelay", true},
+        {&readFromLocal, "dfs.client.read.shortcircuit", true},
+        {&addDatanode, "output.replace-datanode-on-failure", true},
+        {&notRetryAnotherNode, "input.notretry-another-node", false},
+        {&useMappedFile, "input.localread.mappedfile", true}};
     ConfigDefault<int32_t> i32Values[] = {
-        {
-            &rpcMaxIdleTime, "rpc.client.max.idle", 10 * 1000, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &rpcPingTimeout, "rpc.client.ping.interval", 10 * 1000
-        }, {
-            &rpcConnectTimeout, "rpc.client.connect.timeout", 600 * 1000
-        }, {
-            &rpcReadTimeout, "rpc.client.read.timeout", 3600 * 1000
-        }, {
-            &rpcWriteTimeout, "rpc.client.write.timeout", 3600 * 1000
-        }, {
-            &rpcSocketLingerTimeout, "rpc.client.socekt.linger.timeout", -1
-        }, {
-            &rpcMaxRetryOnConnect, "rpc.client.connect.retry", 10, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &rpcTimeout, "rpc.client.timeout", 3600 * 1000
-        }, {
-            &defaultReplica, "dfs.default.replica", 3, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &inputConnTimeout, "input.connect.timeout", 600 * 1000
-        }, {
-            &inputReadTimeout, "input.read.timeout", 3600 * 1000
-        }, {
-            &inputWriteTimeout, "input.write.timeout", 3600 * 1000
-        }, {
-            &localReadBufferSize, "input.localread.default.buffersize", 1 * 1024 * 1024, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &prefetchSize, "dfs.prefetchsize", 10, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &maxGetBlockInfoRetry, "input.read.getblockinfo.retry", 3, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &maxLocalBlockInfoCacheSize, "input.localread.blockinfo.cachesize", 1000, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &maxReadBlockRetry, "input.read.max.retry", 60, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &chunkSize, "output.default.chunksize", 512, bind(CheckMultipleOf<int32_t>, _1, _2, 512)
-        }, {
-            &packetSize, "output.default.packetsize", 64 * 1024
-        }, {
-            &blockWriteRetry, "output.default.write.retry", 10, bind(CheckRangeGE<int32_t>, _1, _2, 1)
-        }, {
-            &outputConnTimeout, "output.connect.timeout", 600 * 1000
-        }, {
-            &outputReadTimeout, "output.read.timeout", 3600 * 1000
-        }, {
-            &outputWriteTimeout, "output.write.timeout", 3600 * 1000
-        }, {
-            &closeFileTimeout, "output.close.timeout", 3600 * 1000
-        }, {
-            &packetPoolSize, "output.packetpool.size", 1024
-        }, {
-            &heartBeatInterval, "output.heeartbeat.interval", 10 * 1000
-        }, {
-            &rpcMaxHARetry, "dfs.client.failover.max.attempts", 15, bind(CheckRangeGE<int32_t>, _1, _2, 0)
-        }
-    };
-    ConfigDefault<int64_t> i64Values [] = {
-        {
-            &defaultBlockSize, "dfs.default.blocksize", 64 * 1024 * 1024, bind(CheckMultipleOf<int64_t>, _1, _2, 512)
-        }
-    };
-    ConfigDefault<std::string> strValues [] = {
-        {&defaultUri, "dfs.default.uri", "hdfs://localhost:9000" },
-        {&rpcAuthMethod, "hadoop.security.authentication", "simple" },
-        {&kerberosCachePath, "hadoop.security.kerberos.ticket.cache.path", "" },
-        {&logSeverity, "dfs.client.log.severity", "INFO" }
-    };
+        {&rpcMaxIdleTime, "rpc.client.max.idle", 10 * 1000,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&rpcPingTimeout, "rpc.client.ping.interval", 10 * 1000},
+        {&rpcConnectTimeout, "rpc.client.connect.timeout", 600 * 1000},
+        {&rpcReadTimeout, "rpc.client.read.timeout", 3600 * 1000},
+        {&rpcWriteTimeout, "rpc.client.write.timeout", 3600 * 1000},
+        {&rpcSocketLingerTimeout, "rpc.client.socekt.linger.timeout", -1},
+        {&rpcMaxRetryOnConnect, "rpc.client.connect.retry", 10,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&rpcTimeout, "rpc.client.timeout", 3600 * 1000},
+        {&defaultReplica, "dfs.default.replica", 3,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&inputConnTimeout, "input.connect.timeout", 600 * 1000},
+        {&inputReadTimeout, "input.read.timeout", 3600 * 1000},
+        {&inputWriteTimeout, "input.write.timeout", 3600 * 1000},
+        {&localReadBufferSize, "input.localread.default.buffersize",
+         1 * 1024 * 1024, bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&prefetchSize, "dfs.prefetchsize", 10,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&maxGetBlockInfoRetry, "input.read.getblockinfo.retry", 3,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&maxLocalBlockInfoCacheSize, "input.localread.blockinfo.cachesize",
+         1000, bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&maxReadBlockRetry, "input.read.max.retry", 60,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&chunkSize, "output.default.chunksize", 512,
+         bind(CheckMultipleOf<int32_t>, _1, _2, 512)},
+        {&packetSize, "output.default.packetsize", 64 * 1024},
+        {&blockWriteRetry, "output.default.write.retry", 10,
+         bind(CheckRangeGE<int32_t>, _1, _2, 1)},
+        {&outputConnTimeout, "output.connect.timeout", 600 * 1000},
+        {&outputReadTimeout, "output.read.timeout", 3600 * 1000},
+        {&outputWriteTimeout, "output.write.timeout", 3600 * 1000},
+        {&closeFileTimeout, "output.close.timeout", 3600 * 1000},
+        {&packetPoolSize, "output.packetpool.size", 1024},
+        {&heartBeatInterval, "output.heeartbeat.interval", 10 * 1000},
+        {&rpcMaxHARetry, "dfs.client.failover.max.attempts", 15,
+         bind(CheckRangeGE<int32_t>, _1, _2, 0)}};
+    ConfigDefault<int64_t> i64Values[] = {
+        {&defaultBlockSize, "dfs.default.blocksize", 64 * 1024 * 1024,
+         bind(CheckMultipleOf<int64_t>, _1, _2, 512)}};
+    ConfigDefault<std::string> strValues[] = {
+        {&defaultUri, "dfs.default.uri", "hdfs://localhost:9000"},
+        {&rpcAuthMethod, "hadoop.security.authentication", "simple"},
+        {&kerberosCachePath, "hadoop.security.kerberos.ticket.cache.path", ""},
+        {&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);
+        *boolValues[i].variable =
+            conf.getBool(boolValues[i].key, boolValues[i].value);
 
         if (boolValues[i].check) {
             boolValues[i].check(boolValues[i].key, *boolValues[i].variable);
@@ -139,8 +113,8 @@ SessionConfig::SessionConfig(const Config & conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(i32Values); ++i) {
-        *i32Values[i].variable = conf.getInt32(i32Values[i].key,
-                                               i32Values[i].value);
+        *i32Values[i].variable =
+            conf.getInt32(i32Values[i].key, i32Values[i].value);
 
         if (i32Values[i].check) {
             i32Values[i].check(i32Values[i].key, *i32Values[i].variable);
@@ -148,8 +122,8 @@ SessionConfig::SessionConfig(const Config & conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(i64Values); ++i) {
-        *i64Values[i].variable = conf.getInt64(i64Values[i].key,
-                                               i64Values[i].value);
+        *i64Values[i].variable =
+            conf.getInt64(i64Values[i].key, i64Values[i].value);
 
         if (i64Values[i].check) {
             i64Values[i].check(i64Values[i].key, *i64Values[i].variable);
@@ -157,14 +131,13 @@ SessionConfig::SessionConfig(const Config & conf) {
     }
 
     for (size_t i = 0; i < ARRAYSIZE(strValues); ++i) {
-        *strValues[i].variable = conf.getString(strValues[i].key,
-                                                strValues[i].value);
+        *strValues[i].variable =
+            conf.getString(strValues[i].key, strValues[i].value.c_str());
 
         if (strValues[i].check) {
             strValues[i].check(strValues[i].key, *strValues[i].variable);
         }
     }
 }
-
 }
 }

+ 9 - 11
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SessionConfig.h

@@ -19,11 +19,11 @@
 #ifndef _HDFS_LIBHDFS3_COMMON_SESSIONCONFIG_H_
 #define _HDFS_LIBHDFS3_COMMON_SESSIONCONFIG_H_
 
+#include "ConfigImpl.h"
 #include "Exception.h"
 #include "ExceptionInternal.h"
 #include "Function.h"
 #include "Logger.h"
-#include "XmlConfig.h"
 
 #include <cassert>
 #include <stdint.h>
@@ -32,18 +32,18 @@
 namespace hdfs {
 namespace internal {
 
-template<typename T>
+template <typename T>
 struct ConfigDefault {
-    T *variable; //variable this configure item should be bound to.
-    const char *key; //configure key.
-    T value; //default value.
-    function<void(const char *, T const &)> check;   //the function to validate the value.
+    T *variable;      // variable this configure item should be bound to.
+    const char *key;  // configure key.
+    T value;          // default value.
+    function<void(const char *, T const &)>
+        check;  // the function to validate the value.
 };
 
 class SessionConfig {
 public:
-
-    SessionConfig(const Config &conf);
+    SessionConfig(const ConfigImpl &conf);
 
     /*
      * rpc configure
@@ -308,16 +308,14 @@ public:
     bool addDatanode;
     int32_t chunkSize;
     int32_t packetSize;
-    int32_t blockWriteRetry; //retry on block not replicated yet.
+    int32_t blockWriteRetry;  // retry on block not replicated yet.
     int32_t outputConnTimeout;
     int32_t outputReadTimeout;
     int32_t outputWriteTimeout;
     int32_t packetPoolSize;
     int32_t heartBeatInterval;
     int32_t closeFileTimeout;
-
 };
-
 }
 }
 

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.cc

@@ -0,0 +1,33 @@
+/**
+ * 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 "Status.h"
+#include "ExceptionInternal.h"
+
+namespace hdfs {
+
+Status::Status() : code(0), msg("Success") {
+}
+
+Status::Status(int code)
+    : code(code), msg(hdfs::internal::GetSystemErrorInfo(code)) {
+}
+
+Status::Status(int code, const std::string &msg) : code(code), msg(msg) {
+}
+}

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

@@ -0,0 +1,60 @@
+/**
+ * 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_STATUS_H_
+#define _HDFS_LIBHDFS3_COMMON_STATUS_H_
+
+#include <string>
+
+namespace hdfs {
+
+class Status {
+public:
+    Status();
+
+    Status(int code);
+
+    Status(int code, const std::string &msg);
+
+    bool operator==(const Status &other) const {
+        return code == other.code;
+    }
+
+    bool isError() const {
+        return code != 0;
+    }
+
+    int getCode() const {
+        return code;
+    }
+
+    const char *getErrorMsg() {
+        return msg.c_str();
+    }
+
+    static Status OK() {
+        return Status();
+    }
+
+private:
+    int code;
+    std::string msg;
+};
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_STATUS_H_ */

+ 137 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StatusInternal.h

@@ -0,0 +1,137 @@
+/**
+ * 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_STATUSINTERNAL_H_
+#define _HDFS_LIBHDFS3_COMMON_STATUSINTERNAL_H_
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "Status.h"
+
+namespace hdfs {
+namespace internal {
+
+#define CHECK_PARAMETER(cond, code, msg) \
+    if (!(cond)) {                       \
+        return Status((code), (msg));    \
+    }
+
+static inline Status CreateStatusFromException(exception_ptr e) {
+    try {
+        hdfs::rethrow_exception(e);
+    } catch (const hdfs::AccessControlException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::AlreadyBeingCreatedException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::ChecksumException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::DSQuotaExceededException &e) {
+        errno = ENOSPC;
+        return Status(errno, e.what());
+    } catch (const hdfs::FileAlreadyExistsException &e) {
+        errno = EEXIST;
+        return Status(errno, e.what());
+    } catch (const hdfs::FileNotFoundException &e) {
+        errno = ENOENT;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsBadBoolFoumat &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsBadConfigFoumat &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsBadNumFoumat &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsCanceled &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsConfigInvalid &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsConfigNotFound &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsEndOfStream &e) {
+        errno = EOVERFLOW;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsInvalidBlockToken &e) {
+        errno = EPERM;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsTimeoutException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::InvalidParameter &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::InvalidPath &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::NotReplicatedYetException &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::NSQuotaExceededException &e) {
+        errno = EINVAL;
+        return Status(errno, e.what());
+    } catch (const hdfs::ParentNotDirectoryException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::ReplicaNotFoundException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::SafeModeException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::UnresolvedLinkException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsRpcException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsNetworkException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::RpcNoSuchMethodException &e) {
+        errno = ENOTSUP;
+        return Status(errno, e.what());
+    } catch (const hdfs::SaslException &e) {
+        errno = EACCES;
+        return Status(errno, e.what());
+    } catch (const hdfs::NameNodeStandbyException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsIOException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (const hdfs::HdfsException &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    } catch (std::exception &e) {
+        errno = EIO;
+        return Status(errno, e.what());
+    }
+
+    return Status(EIO, "Unknown Error");
+}
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_STATUSINTERNAL_H_ */

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h

@@ -22,24 +22,26 @@
 #ifdef _LIBCPP_VERSION
 
 #include <unordered_map>
+#include <unordered_set>
 
 namespace hdfs {
 namespace internal {
 
+using std::unordered_set;
 using std::unordered_map;
-
 }
 }
 
 #else
 
 #include <tr1/unordered_map>
+#include <tr1/unordered_set>
 
 namespace hdfs {
 namespace internal {
 
+using std::tr1::unordered_set;
 using std::tr1::unordered_map;
-
 }
 }
 

+ 0 - 395
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfig.cc

@@ -1,395 +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 "XmlConfig.h"
-
-#include <cassert>
-#include <errno.h>
-#include <fstream>
-#include <libxml/parser.h>
-#include <libxml/tree.h>
-#include <limits>
-#include <string.h>
-#include <unistd.h>
-#include <vector>
-
-using namespace hdfs::internal;
-
-using std::map;
-using std::string;
-using std::vector;
-
-namespace hdfs {
-
-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;
-}
-
-static void readConfigItem(xmlNodePtr root, Map & kv, const char *path) {
-    std::string key, value;
-    xmlNodePtr curNode;
-    bool hasname = false, hasvalue = false;
-
-    for (curNode = root; NULL != curNode; curNode = curNode->next) {
-        if (curNode->type != XML_ELEMENT_NODE) {
-            continue;
-        }
-
-        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;
-            }
-        } 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;
-            }
-        } else {
-            continue;
-        }
-    }
-
-    if (hasname && hasvalue) {
-        kv[key] = value;
-        return;
-    } else if (hasname) {
-        kv[key] = "";
-        return;
-    }
-
-    THROW(HdfsBadConfigFoumat, "Config cannot parse configure file: \"%s\"",
-          path);
-}
-
-static void readConfigItems(xmlDocPtr doc, Map & kv, const char *path) {
-    xmlNodePtr root, curNode;
-    root = xmlDocGetRootElement(doc);
-
-    if (NULL == root || strcmp((const char *) root->name, "configuration")) {
-        THROW(HdfsBadConfigFoumat, "Config cannot parse configure file: \"%s\"",
-              path);
-    }
-
-    /*
-     * for each property
-     */
-    for (curNode = root->children; NULL != curNode; curNode = curNode->next) {
-        if (curNode->type != XML_ELEMENT_NODE) {
-            continue;
-        }
-
-        if (strcmp((const char *) curNode->name, "property")) {
-            THROW(HdfsBadConfigFoumat,
-                  "Config cannot parse configure file: \"%s\"", path);
-        }
-
-        readConfigItem(curNode->children, kv, path);
-    }
-}
-
-Config::Config(const char *p) :
-    path(p) {
-    update(p);
-}
-
-void Config::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 configure file: \"%s\", %s",
-              path.c_str(), msg);
-    }
-
-    /* parse the file */
-    doc = xmlReadFile(path.c_str(), NULL, 0);
-
-    try {
-        /* check if parsing succeeded */
-        if (doc == NULL) {
-            THROW(HdfsBadConfigFoumat,
-                  "Config cannot parse configure file: \"%s\"", path.c_str());
-        } else {
-            readConfigItems(doc, kv, path.c_str());
-            /* free up the resulting document */
-            xmlFreeDoc(doc);
-        }
-    } catch (...) {
-        xmlFreeDoc(doc);
-        throw;
-    }
-}
-
-const char *Config::getString(const char *key) const {
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return it->second.c_str();
-}
-
-const char *Config::getString(const char *key, const char *def) const {
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        return def;
-    } else {
-        return it->second.c_str();
-    }
-}
-
-const char *Config::getString(const std::string & key) const {
-    return getString(key.c_str());
-}
-
-const char *Config::getString(const std::string & key,
-                               const std::string & def) const {
-    return getString(key.c_str(), def.c_str());
-}
-
-int64_t Config::getInt64(const char *key) const {
-    int64_t retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    try {
-        retval = StrToInt64(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-int64_t Config::getInt64(const char *key, int64_t def) const {
-    int64_t retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToInt64(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-int32_t Config::getInt32(const char *key) const {
-    int32_t retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    try {
-        retval = StrToInt32(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-int32_t Config::getInt32(const char *key, int32_t def) const {
-    int32_t retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToInt32(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-double Config::getDouble(const char *key) const {
-    double retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    try {
-        retval = StrToDouble(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-double Config::getDouble(const char *key, double def) const {
-    double retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToDouble(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-bool Config::getBool(const char *key) const {
-    bool retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    try {
-        retval = StrToBool(it->second.c_str());
-    } catch (const HdfsBadBoolFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-bool Config::getBool(const char *key, bool def) const {
-    bool retval;
-    Iterator it = kv.find(key);
-
-    if (kv.end() == it) {
-        return def;
-    }
-
-    try {
-        retval = StrToBool(it->second.c_str());
-    } catch (const HdfsBadNumFoumat & e) {
-        NESTED_THROW(HdfsConfigNotFound, "Config key: %s not found", key);
-    }
-
-    return retval;
-}
-
-size_t Config::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());
-}
-
-}
-

+ 154 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.cc

@@ -0,0 +1,154 @@
+/**
+ * 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 "XmlConfigParser.h"
+
+#include <cassert>
+#include <errno.h>
+#include <fstream>
+#include <libxml/parser.h>
+#include <libxml/tree.h>
+#include <limits>
+#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;
+
+static void readXmlConfigParserItem(xmlNodePtr root, Map &kv,
+                                    const char *path) {
+    std::string key, value;
+    xmlNodePtr curNode;
+    bool hasname = false, hasvalue = false;
+
+    for (curNode = root; NULL != curNode; curNode = curNode->next) {
+        if (curNode->type != XML_ELEMENT_NODE) {
+            continue;
+        }
+
+        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;
+            }
+        } 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;
+            }
+        } else {
+            continue;
+        }
+    }
+
+    if (hasname && hasvalue) {
+        kv[key] = value;
+        return;
+    } else if (hasname) {
+        kv[key] = "";
+        return;
+    }
+
+    THROW(HdfsBadConfigFoumat,
+          "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"", path);
+}
+
+static void readXmlConfigParserItems(xmlDocPtr doc, Map &kv, const char *path) {
+    xmlNodePtr root, curNode;
+    root = xmlDocGetRootElement(doc);
+
+    if (NULL == root ||
+        strcmp((const char *)root->name, "XmlConfigParseruration")) {
+        THROW(HdfsBadConfigFoumat,
+              "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"",
+              path);
+    }
+
+    /*
+     * for each property
+     */
+    for (curNode = root->children; NULL != curNode; curNode = curNode->next) {
+        if (curNode->type != XML_ELEMENT_NODE) {
+            continue;
+        }
+
+        if (strcmp((const char *)curNode->name, "property")) {
+            THROW(
+                HdfsBadConfigFoumat,
+                "XmlConfigParser cannot parse XmlConfigParserure file: \"%s\"",
+                path);
+        }
+
+        readXmlConfigParserItem(curNode->children, kv, path);
+    }
+}
+
+XmlConfigParser::XmlConfigParser(const char *p) : path(p) {
+    update(p);
+}
+
+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);
+    }
+
+    /* 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);
+        }
+    } catch (...) {
+        xmlFreeDoc(doc);
+        throw;
+    }
+}
+}
+}

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/XmlConfigParser.h

@@ -0,0 +1,69 @@
+/**
+ * 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_XMLCONFIGPARSER_H_
+#define _HDFS_LIBHDFS3_COMMON_XMLCONFIGPARSER_H_
+
+#include <stdint.h>
+#include <string>
+#include <sstream>
+#include <map>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A configure file parser.
+ */
+class XmlConfigParser {
+public:
+    /**
+     * Construct a empty Config instance.
+     */
+    XmlConfigParser() {
+    }
+
+    /**
+     * Construct a Config with given configure file.
+     * @param path The path of configure file.
+     * @throw HdfsBadConfigFoumat
+     */
+    XmlConfigParser(const char *path);
+
+    /**
+     * Parse the configure file.
+     * @throw HdfsBadConfigFoumat
+     */
+    void update(const char *path);
+
+    /**
+     * Get Key Values
+     * @return Return the Key Value pairs.
+     */
+    std::map<std::string, std::string> getKeyValue() {
+        return kv;
+    }
+
+private:
+    std::string path;
+    std::map<std::string, std::string> kv;
+};
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_COMMON_XMLCONFIGPARSER_H_ */

+ 6 - 13
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc

@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 #include "Exception.h"
 #include "ExceptionInternal.h"
 #include "LocatedBlock.h"
@@ -28,22 +27,19 @@
 namespace hdfs {
 namespace internal {
 
-const LocatedBlock *LocatedBlocksImpl::findBlock(int64_t position) {
+const LocatedBlock *LocatedBlocks::findBlock(int64_t position) {
     if (position < fileLength) {
         LocatedBlock target(position);
         std::vector<LocatedBlock>::iterator bound;
-
         if (blocks.empty()) {
             return NULL;
         }
-
-        /*
-         * up is the first block which offset is not less than position.
-         */
+        // Find first block whose offset is equal to or greater than the
+        // requested position.
         bound = std::lower_bound(blocks.begin(), blocks.end(), target,
                                  std::less<LocatedBlock>());
         assert(bound == blocks.end() || bound->getOffset() >= position);
-        LocatedBlock * retval = NULL;
+        LocatedBlock *retval = NULL;
 
         if (bound == blocks.end()) {
             retval = &blocks.back();
@@ -53,17 +49,14 @@ const LocatedBlock *LocatedBlocksImpl::findBlock(int64_t position) {
         } else {
             retval = &(*bound);
         }
-
-        if (position < retval->getOffset()
-                || position >= retval->getOffset() + retval->getNumBytes()) {
+        if (position < retval->getOffset() ||
+            position >= retval->getOffset() + retval->getNumBytes()) {
             return NULL;
         }
-
         return retval;
     } else {
         return lastBlock.get();
     }
 }
-
 }
 }

+ 3 - 34
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h

@@ -20,43 +20,14 @@
 #define _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCKS_H_
 
 #include "LocatedBlock.h"
-#include "common/SharedPtr.h"
+#include "SharedPtr.h"
 
 #include <cassert>
-#include <vector>
 
 namespace hdfs {
 namespace internal {
 
 class LocatedBlocks {
-public:
-    virtual ~LocatedBlocks() {}
-
-    virtual int64_t getFileLength() const = 0;
-
-    virtual void setFileLength(int64_t fileLength) = 0;
-
-    virtual bool isLastBlockComplete() const = 0;
-
-    virtual void setIsLastBlockComplete(bool lastBlockComplete) = 0;
-
-    virtual shared_ptr<LocatedBlock> getLastBlock() = 0;
-
-    virtual void setLastBlock(shared_ptr<LocatedBlock> lastBlock) = 0;
-
-    virtual bool isUnderConstruction() const = 0;
-
-    virtual void setUnderConstruction(bool underConstruction) = 0;
-
-    virtual const LocatedBlock *findBlock(int64_t position) = 0;
-
-    virtual std::vector<LocatedBlock> &getBlocks() = 0;
-};
-
-/**
- * Collection of blocks with their locations and the file length.
- */
-class LocatedBlocksImpl : public LocatedBlocks {
 public:
     int64_t getFileLength() const {
         return fileLength;
@@ -91,9 +62,9 @@ public:
         this->underConstruction = underConstruction;
     }
 
-    const LocatedBlock * findBlock(int64_t position);
+    const LocatedBlock *findBlock(int64_t position);
 
-    std::vector<LocatedBlock> & getBlocks() {
+    std::vector<LocatedBlock> &getBlocks() {
         return blocks;
     }
 
@@ -103,9 +74,7 @@ private:
     int64_t fileLength;
     shared_ptr<LocatedBlock> lastBlock;
     std::vector<LocatedBlock> blocks;
-
 };
-
 }
 }
 #endif /* _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCKS_H_ */

+ 123 - 87
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h

@@ -42,6 +42,9 @@ namespace internal {
 
 class Namenode {
 public:
+    Namenode() {
+    }
+
     /**
      * Destroy the namenode.
      */
@@ -49,7 +52,8 @@ public:
     }
 
     /**
-     * Get locations of the blocks of the specified file within the specified range.
+     * Get locations of the blocks of the specified file within the specified
+     *range.
      * DataNode locations for each block are sorted by
      * the proximity to the client.
      * <p>
@@ -72,10 +76,10 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual void getBlockLocations(const std::string &src, int64_t offset,
                                    int64_t length, LocatedBlocks &lbs)
-      /* throw (AccessControlException,
+        /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */ = 0;
 
@@ -86,20 +90,27 @@ public:
      * The path should reflect a full path originated at the root.
      * The name-node does not have a notion of "current" directory for a client.
      * <p>
-     * Once created, the file is visible and available for read to other clients.
-     * Although, other clients cannot {//link #delete(const std::string &, bool)}, re-create or
-     * {//link #rename(const std::string &, const std::string &)} it until the file is completed
+     * Once created, the file is visible and available for read to other
+     *clients.
+     * Although, other clients cannot {//link #delete(const std::string &,
+     *bool)},
+     *re-create or
+     * {//link #rename(const std::string &, const std::string &)} it until the
+     *file is completed
      * or explicitly as a result of lease expiration.
      * <p>
      * Blocks have a maximum size.  Clients that intend to create
      * multi-block files must also use
-     * {//link #addBlock(const std::string &, const std::string &, ExtendedBlock, DatanodeInfo[])}
+     * {//link #addBlock(const std::string &, const std::string &,
+     *ExtendedBlock,
+     *DatanodeInfo[])}
      *
      * @param src path of the file being created.
      * @param masked masked permission.
      * @param clientName name of the current client.
      * @param flag indicates whether the file should be
-     * overwritten if it already exists or create if it does not exist or append.
+     * overwritten if it already exists or create if it does not exist or
+     *append.
      * @param createParent create missing parent directory if true
      * @param replication block replication factor.
      * @param blockSize maximum block size.
@@ -123,9 +134,9 @@ public:
      * @throw InvalidPathException Path <code>src</code> is invalid
      */
     virtual void create(const std::string &src, const Permission &masked,
-                        const std::string &clientName, int flag, bool createParent,
-                        short replication, int64_t blockSize)
-      /* throw (AccessControlException,
+                        const std::string &clientName, int flag,
+                        bool createParent, short replication, int64_t blockSize)
+        /* throw (AccessControlException,
              AlreadyBeingCreatedException, DSQuotaExceededException,
              FileAlreadyExistsException, FileNotFoundException,
              NSQuotaExceededException, ParentNotDirectoryException,
@@ -158,7 +169,7 @@ public:
      */
     virtual shared_ptr<LocatedBlock> append(const std::string &src,
                                             const std::string &clientName)
-    /* throw (AccessControlException,
+        /* throw (AccessControlException,
              DSQuotaExceededException, FileNotFoundException,
              SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
 
@@ -184,9 +195,9 @@ public:
      * @throw UnresolvedLinkException if <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual bool setReplication(const std::string &src, short replication)
-    /* throw (AccessControlException, DSQuotaExceededException,
+        /* throw (AccessControlException, DSQuotaExceededException,
      FileNotFoundException, SafeModeException, UnresolvedLinkException,
      HdfsIOException) */ = 0;
 
@@ -199,9 +210,10 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
-    virtual void setPermission(const std::string &src,
-                               const Permission &permission) /* throw (AccessControlException,
+    // Idempotent
+    virtual void setPermission(
+        const std::string &src,
+        const Permission &permission) /* throw (AccessControlException,
              FileNotFoundException, SafeModeException,
              UnresolvedLinkException, HdfsIOException) */ = 0;
 
@@ -218,9 +230,10 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
-    virtual void setOwner(const std::string &src, const std::string &username,
-                          const std::string &groupname) /* throw (AccessControlException,
+    // Idempotent
+    virtual void setOwner(
+        const std::string &src, const std::string &username,
+        const std::string &groupname) /* throw (AccessControlException,
              FileNotFoundException, SafeModeException,
              UnresolvedLinkException, HdfsIOException) */ = 0;
 
@@ -235,8 +248,9 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    virtual void abandonBlock(const ExtendedBlock &b, const std::string &src,
-                              const std::string &holder) /* throw (AccessControlException,
+    virtual void abandonBlock(
+        const ExtendedBlock &b, const std::string &src,
+        const std::string &holder) /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */ = 0;
 
@@ -270,10 +284,11 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    virtual shared_ptr<LocatedBlock> addBlock(const std::string &src,
-            const std::string &clientName, const ExtendedBlock *previous,
-            const std::vector<DatanodeInfo> &excludeNodes)
-    /* throw (AccessControlException, FileNotFoundException,
+    virtual shared_ptr<LocatedBlock> addBlock(
+        const std::string &src, const std::string &clientName,
+        const ExtendedBlock *previous,
+        const std::vector<DatanodeInfo> &excludeNodes)
+        /* throw (AccessControlException, FileNotFoundException,
      NotReplicatedYetException, SafeModeException,
      UnresolvedLinkException, HdfsIOException) */ = 0;
 
@@ -296,14 +311,14 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
-    virtual shared_ptr<LocatedBlock> getAdditionalDatanode(const std::string &src,
-            const ExtendedBlock &blk,
-            const std::vector<DatanodeInfo> &existings,
-            const std::vector<std::string> &storageIDs,
-            const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
-            const std::string &clientName)
-    /* throw (AccessControlException, FileNotFoundException,
+    // Idempotent
+    virtual shared_ptr<LocatedBlock> getAdditionalDatanode(
+        const std::string &src, const ExtendedBlock &blk,
+        const std::vector<DatanodeInfo> &existings,
+        const std::vector<std::string> &storageIDs,
+        const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
+        const std::string &clientName)
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
@@ -328,9 +343,9 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    virtual bool complete(const std::string &src,
-                          const std::string &clientName, const ExtendedBlock *last)
-    /* throw (AccessControlException, FileNotFoundException,
+    virtual bool complete(const std::string &src, const std::string &clientName,
+                          const ExtendedBlock *last)
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
@@ -338,7 +353,7 @@ public:
      * locations on datanodes).
      * @param blocks Array of located blocks to report
      */
-    //Idempotent
+    // Idempotent
     /*    virtual void reportBadBlocks(const std::vector<LocatedBlock> &blocks)
          throw (HdfsIOException)  = 0;*/
 
@@ -352,7 +367,7 @@ public:
      * @throw HdfsIOException an I/O error occurred
      */
     virtual bool rename(const std::string &src, const std::string &dst)
-    /* throw (UnresolvedLinkException, HdfsIOException) */ = 0;
+        /* throw (UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
      * Moves blocks from srcs to trg and delete srcs
@@ -364,7 +379,8 @@ public:
      *           contains a symlink
      */
     /*    virtual void concat(const std::string &trg,
-                            const std::vector<std::string> &srcs)  throw (HdfsIOException,
+                            const std::vector<std::string> &srcs)  throw
+       (HdfsIOException,
                  UnresolvedLinkException)  = 0;*/
 
     /**
@@ -425,7 +441,7 @@ public:
      * @throw HdfsIOException If an I/O error occurred
      */
     virtual bool deleteFile(const std::string &src, bool recursive)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
@@ -442,7 +458,8 @@ public:
      * @throw FileAlreadyExistsException If <code>src</code> already exists
      * @throw FileNotFoundException If parent of <code>src</code> does not exist
      *           and <code>createParent</code> is false
-     * @throw NSQuotaExceededException If file creation violates quota restriction
+     * @throw NSQuotaExceededException If file creation violates quota
+     *restriction
      * @throw ParentNotDirectoryException If parent of <code>src</code>
      *           is not a directory
      * @throw SafeModeException create not allowed in safemode
@@ -452,9 +469,10 @@ public:
      * RunTimeExceptions:
      * @throw InvalidPathException If <code>src</code> is invalid
      */
-    //Idempotent
-    virtual bool mkdirs(const std::string &src, const Permission &masked,
-                        bool createParent) /* throw (AccessControlException,
+    // Idempotent
+    virtual bool mkdirs(
+        const std::string &src, const Permission &masked,
+        bool createParent) /* throw (AccessControlException,
              FileAlreadyExistsException, FileNotFoundException,
              NSQuotaExceededException, ParentNotDirectoryException,
              SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
@@ -474,10 +492,11 @@ public:
      * @throw UnresolvedLinkException If <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
-    virtual bool getListing(const std::string &src,
-                            const std::string &startAfter, bool needLocation,
-                            std::vector<FileStatus> &dl) /* throw (AccessControlException,
+    // Idempotent
+    virtual bool getListing(
+        const std::string &src, const std::string &startAfter,
+        bool needLocation,
+        std::vector<FileStatus> &dl) /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */ = 0;
 
@@ -500,9 +519,9 @@ public:
      * @throw AccessControlException permission denied
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual void renewLease(const std::string &clientName)
-    /* throw (AccessControlException, HdfsIOException) */ = 0;
+        /* throw (AccessControlException, HdfsIOException) */ = 0;
 
     /**
      * Start lease recovery.
@@ -513,7 +532,7 @@ public:
      * @param true if the file is already closed
      * @throw HdfsIOException
      */
-    //Idempotent
+    // Idempotent
     virtual bool recoverLease(const std::string &src,
                               const std::string &clientName) = 0;
 
@@ -521,7 +540,8 @@ public:
      * Get a set of statistics about the filesystem.
      * Right now, only seven values are returned.
      * <ul>
-     * <li> [0] contains the total storage capacity of the system, in bytes.</li>
+     * <li> [0] contains the total storage capacity of the system, in
+     * bytes.</li>
      * <li> [1] contains the total used space of the system, in bytes.</li>
      * <li> [2] contains the available storage of the system, in bytes.</li>
      * <li> [3] contains number of under replicated blocks in the system.</li>
@@ -531,7 +551,7 @@ public:
      * Use  constants like {//link #GET_STATS_CAPACITY_IDX} in place of
      * actual numbers to index into the array.
      */
-    //Idempotent
+    // Idempotent
     virtual std::vector<int64_t> getFsStats() /* throw (HdfsIOException) */ = 0;
 
     /**
@@ -554,9 +574,9 @@ public:
      * @throw UnresolvedLinkException if the path contains a symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual FileStatus getFileInfo(const std::string &src)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
@@ -571,7 +591,7 @@ public:
      * @throw UnresolvedLinkException if <code>src</code> contains a symlink
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     /*    virtual FileStatus getFileLinkInfo(const std::string &src)
          throw (AccessControlException, UnresolvedLinkException,
          HdfsIOException)  = 0;*/
@@ -585,14 +605,15 @@ public:
      * @throw UnresolvedLinkException if <code>path</code> contains a symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     /*    virtual ContentSummary getContentSummary(const std::string &path)
          throw (AccessControlException, FileNotFoundException,
          UnresolvedLinkException, HdfsIOException)  = 0;*/
 
     /**
      * Set the quota for a directory.
-     * @param path  The const std::string &representation of the path to the directory
+     * @param path  The const std::string &representation of the path to the
+     *directory
      * @param namespaceQuota Limit on the number of names in the tree rooted
      *                       at the directory
      * @param diskspaceQuota Limit on disk space occupied all the files under
@@ -600,7 +621,8 @@ public:
      * <br><br>
      *
      * The quota can have three types of values : (1) 0 or more will set
-     * the quota to that value, (2) {//link HdfsConstants#QUOTA_DONT_SET}  implies
+     * the quota to that value, (2) {//link HdfsConstants#QUOTA_DONT_SET}
+     *implies
      * the quota will not be changed, and (3) {//link HdfsConstants#QUOTA_RESET}
      * implies the quota will be reset. Any other value is a runtime error.
      *
@@ -608,13 +630,15 @@ public:
      * @throw FileNotFoundException file <code>path</code> is not found
      * @throw QuotaExceededException if the directory size
      *           is greater than the given quota
-     * @throw UnresolvedLinkException if the <code>path</code> contains a symlink.
+     * @throw UnresolvedLinkException if the <code>path</code> contains a
+     *symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     /*
         virtual void setQuota(const std::string &path, int64_t namespaceQuota,
-                              int64_t diskspaceQuota)  throw (AccessControlException,
+                              int64_t diskspaceQuota)  throw
+       (AccessControlException,
                  FileNotFoundException, UnresolvedLinkException,
                  HdfsIOException)  = 0;
     */
@@ -630,16 +654,18 @@ public:
      * @throw UnresolvedLinkException if <code>src</code> contains a symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual void fsync(const std::string &src, const std::string &client)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
      * Sets the modification and access time of the file to the specified time.
      * @param src The const std::string &representation of the path
      * @param mtime The number of milliseconds since Jan 1, 1970.
-     *              Setting mtime to -1 means that modification time should not be set
+     *              Setting mtime to -1 means that modification time should not
+     *be
+     *set
      *              by this call.
      * @param atime The number of milliseconds since Jan 1, 1970.
      *              Setting atime to -1 means that access time should not be set
@@ -650,9 +676,9 @@ public:
      * @throw UnresolvedLinkException if <code>src</code> contains a symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     virtual void setTimes(const std::string &src, int64_t mtime, int64_t atime)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */ = 0;
 
     /**
@@ -665,17 +691,22 @@ public:
      *                       if false then parent must exist
      *
      * @throw AccessControlException permission denied
-     * @throw FileAlreadyExistsException If file <code>link</code> already exists
-     * @throw FileNotFoundException If parent of <code>link</code> does not exist
+     * @throw FileAlreadyExistsException If file <code>link</code> already
+     *exists
+     * @throw FileNotFoundException If parent of <code>link</code> does not
+     *exist
      *           and <code>createParent</code> is false
-     * @throw ParentNotDirectoryException If parent of <code>link</code> is not a
+     * @throw ParentNotDirectoryException If parent of <code>link</code> is not
+     *a
      *           directory.
      * @throw UnresolvedLinkException if <code>link</target> contains a symlink.
      * @throw HdfsIOException If an I/O error occurred
      */
     /*    virtual void createSymlink(const std::string &target,
-                                   const std::string &link, const Permission &dirPerm,
-                                   bool createParent)  throw (AccessControlException,
+                                   const std::string &link, const Permission
+       &dirPerm,
+                                   bool createParent)  throw
+       (AccessControlException,
                  FileAlreadyExistsException, FileNotFoundException,
                  ParentNotDirectoryException, SafeModeException,
                  UnresolvedLinkException, HdfsIOException)  = 0;*/
@@ -692,7 +723,7 @@ public:
      * @throw HdfsIOException If the given path does not refer to a symlink
      *           or an I/O error occurred
      */
-    //Idempotent
+    // Idempotent
     /*    virtual std::string getLinkTarget(const std::string &path)
          throw (AccessControlException, FileNotFoundException,
          HdfsIOException)  = 0;*/
@@ -710,10 +741,10 @@ public:
      * @param lb output the returned block.
      * @throw HdfsIOException if any error occurs
      */
-    //Idempotent
-    virtual shared_ptr<LocatedBlock> updateBlockForPipeline(const ExtendedBlock &block,
-            const std::string &clientName)
-    /* throw (HdfsIOException) */ = 0;
+    // Idempotent
+    virtual shared_ptr<LocatedBlock> updateBlockForPipeline(
+        const ExtendedBlock &block, const std::string &clientName)
+        /* throw (HdfsIOException) */ = 0;
 
     /**
      * Update a pipeline for a block under construction
@@ -724,10 +755,12 @@ public:
      * @param newNodes datanodes in the pipeline
      * @throw HdfsIOException if any error occurs
      */
-    virtual void updatePipeline(const std::string &clientName,
-                                const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
-                                const std::vector<DatanodeInfo> &newNodes,
-                                const std::vector<std::string> &storageIDs) /* throw (HdfsIOException) */ = 0;
+    virtual void updatePipeline(
+        const std::string &clientName, const ExtendedBlock &oldBlock,
+        const ExtendedBlock &newBlock,
+        const std::vector<DatanodeInfo> &newNodes,
+        const std::vector<std::string> &
+            storageIDs) /* throw (HdfsIOException) */ = 0;
 
     /**
       * Get a valid Delegation Token.
@@ -737,7 +770,7 @@ public:
       * @throws IOException
       */
     virtual Token getDelegationToken(const std::string &renewer)
-    /* throws IOException*/ = 0;
+        /* throws IOException*/ = 0;
 
     /**
      * Renew an existing delegation token.
@@ -747,7 +780,7 @@ public:
      * @throws IOException
      */
     virtual int64_t renewDelegationToken(const Token &token)
-    /*throws IOException*/ = 0;
+        /*throws IOException*/ = 0;
 
     /**
      * Cancel an existing delegation token.
@@ -756,14 +789,17 @@ public:
      * @throws IOException
      */
     virtual void cancelDelegationToken(const Token &token)
-    /*throws IOException*/ = 0;
+        /*throws IOException*/ = 0;
 
     /**
      * close the namenode connection.
      */
-    virtual void close() {};
-};
+    virtual void close(){};
 
+private:
+    Namenode(const Namenode &other);
+    Namenode &operator=(const Namenode &other);
+};
 }
 }
 

+ 77 - 69
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h

@@ -24,70 +24,76 @@
 namespace hdfs {
 namespace internal {
 
-class NamenodeImpl: public Namenode {
+class NamenodeImpl : public Namenode {
 public:
     NamenodeImpl(const char *host, const char *port,
-            const std::string &tokenService, const SessionConfig &c,
-            const RpcAuth &a);
+                 const std::string &tokenService, const SessionConfig &c,
+                 const RpcAuth &a);
 
     ~NamenodeImpl();
 
-    //Idempotent
-    void getBlockLocations(const std::string &src, int64_t offset,
-                           int64_t length, LocatedBlocks &lbs) /* throw (AccessControlException,
+    // Idempotent
+    void getBlockLocations(
+        const std::string &src, int64_t offset, int64_t length,
+        LocatedBlocks &lbs) /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */;
 
     void create(const std::string &src, const Permission &masked,
                 const std::string &clientName, int flag, bool createParent,
-                short replication, int64_t blockSize) /* throw (AccessControlException,
+                short replication,
+                int64_t blockSize) /* throw (AccessControlException,
              AlreadyBeingCreatedException, DSQuotaExceededException,
              FileAlreadyExistsException, FileNotFoundException,
              NSQuotaExceededException, ParentNotDirectoryException,
              SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    shared_ptr<LocatedBlock> append(const std::string &src, const std::string &clientName)
-    /* throw (AccessControlException,
+    shared_ptr<LocatedBlock> append(const std::string &src,
+                                    const std::string &clientName)
+        /* throw (AccessControlException,
              DSQuotaExceededException, FileNotFoundException,
              SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     bool setReplication(const std::string &src, short replication)
-    /* throw (AccessControlException, DSQuotaExceededException,
+        /* throw (AccessControlException, DSQuotaExceededException,
      FileNotFoundException, SafeModeException, UnresolvedLinkException,
      HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     void setPermission(const std::string &src, const Permission &permission)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
-    void setOwner(const std::string &src, const std::string &username,
-                  const std::string &groupname) /* throw (AccessControlException,
+    // Idempotent
+    void setOwner(
+        const std::string &src, const std::string &username,
+        const std::string &groupname) /* throw (AccessControlException,
              FileNotFoundException, SafeModeException,
              UnresolvedLinkException, HdfsIOException) */;
 
-    void abandonBlock(const ExtendedBlock &b, const std::string &src,
-                      const std::string &holder) /* throw (AccessControlException,
+    void abandonBlock(
+        const ExtendedBlock &b, const std::string &src,
+        const std::string &holder) /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */;
 
-    shared_ptr<LocatedBlock> addBlock(const std::string &src, const std::string &clientName,
-                                      const ExtendedBlock *previous,
-                                      const std::vector<DatanodeInfo> &excludeNodes)
-    /* throw (AccessControlException, FileNotFoundException,
+    shared_ptr<LocatedBlock> addBlock(
+        const std::string &src, const std::string &clientName,
+        const ExtendedBlock *previous,
+        const std::vector<DatanodeInfo> &excludeNodes)
+        /* throw (AccessControlException, FileNotFoundException,
      NotReplicatedYetException, SafeModeException,
      UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
-    shared_ptr<LocatedBlock> getAdditionalDatanode(const std::string &src,
-            const ExtendedBlock &blk,
-            const std::vector<DatanodeInfo> &existings,
-            const std::vector<std::string> &storageIDs,
-            const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
-            const std::string &clientName)
-    /* throw (AccessControlException, FileNotFoundException,
+    // Idempotent
+    shared_ptr<LocatedBlock> getAdditionalDatanode(
+        const std::string &src, const ExtendedBlock &blk,
+        const std::vector<DatanodeInfo> &existings,
+        const std::vector<std::string> &storageIDs,
+        const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
+        const std::string &clientName)
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
     bool complete(const std::string &src, const std::string &clientName,
@@ -95,15 +101,15 @@ public:
              FileNotFoundException, SafeModeException,
              UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     void reportBadBlocks(const std::vector<LocatedBlock> &blocks)
-    /* throw (HdfsIOException) */;
+        /* throw (HdfsIOException) */;
 
     bool rename(const std::string &src, const std::string &dst)
-    /* throw (UnresolvedLinkException, HdfsIOException) */;
+        /* throw (UnresolvedLinkException, HdfsIOException) */;
 
     void concat(const std::string &trg, const std::vector<std::string> &srcs)
-    /* throw (HdfsIOException, UnresolvedLinkException) */;
+        /* throw (HdfsIOException, UnresolvedLinkException) */;
 
     /*void rename2(const std::string &src, const std::string &dst)
      throw (AccessControlException, DSQuotaExceededException,
@@ -112,43 +118,43 @@ public:
      SafeModeException, UnresolvedLinkException, HdfsIOException) ;*/
 
     bool deleteFile(const std::string &src, bool recursive)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     bool mkdirs(const std::string &src, const Permission &masked,
                 bool createParent) /* throw (AccessControlException,
              FileAlreadyExistsException, FileNotFoundException,
              NSQuotaExceededException, ParentNotDirectoryException,
              SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     bool getListing(const std::string &src, const std::string &startAfter,
                     bool needLocation, std::vector<FileStatus> &dl)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     void renewLease(const std::string &clientName)
-    /* throw (AccessControlException, HdfsIOException) */;
+        /* throw (AccessControlException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     bool recoverLease(const std::string &src, const std::string &clientName)
-    /* throw (HdfsIOException) */;
+        /* throw (HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     std::vector<int64_t> getFsStats() /* throw (HdfsIOException) */;
 
     void metaSave(const std::string &filename) /* throw (HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     FileStatus getFileInfo(const std::string &src)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     FileStatus getFileLinkInfo(const std::string &src)
-    /* throw (AccessControlException, UnresolvedLinkException,
+        /* throw (AccessControlException, UnresolvedLinkException,
      HdfsIOException) */;
 
     /*    //Idempotent
@@ -156,66 +162,68 @@ public:
          throw (AccessControlException, FileNotFoundException,
          UnresolvedLinkException, HdfsIOException) ;*/
 
-    //Idempotent
+    // Idempotent
     void setQuota(const std::string &path, int64_t namespaceQuota,
                   int64_t diskspaceQuota) /* throw (AccessControlException,
              FileNotFoundException, UnresolvedLinkException,
              HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     void fsync(const std::string &src, const std::string &client)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     void setTimes(const std::string &src, int64_t mtime, int64_t atime)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      UnresolvedLinkException, HdfsIOException) */;
 
     void createSymlink(const std::string &target, const std::string &link,
                        const Permission &dirPerm, bool createParent)
-    /* throw (AccessControlException, FileAlreadyExistsException,
+        /* throw (AccessControlException, FileAlreadyExistsException,
      FileNotFoundException, ParentNotDirectoryException,
      SafeModeException, UnresolvedLinkException, HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     std::string getLinkTarget(const std::string &path)
-    /* throw (AccessControlException, FileNotFoundException,
+        /* throw (AccessControlException, FileNotFoundException,
      HdfsIOException) */;
 
-    //Idempotent
-    shared_ptr<LocatedBlock> updateBlockForPipeline(const ExtendedBlock &block,
-            const std::string &clientName)
-    /* throw (HdfsIOException) */;
+    // Idempotent
+    shared_ptr<LocatedBlock> updateBlockForPipeline(
+        const ExtendedBlock &block, const std::string &clientName)
+        /* throw (HdfsIOException) */;
 
     void updatePipeline(const std::string &clientName,
-                        const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
+                        const ExtendedBlock &oldBlock,
+                        const ExtendedBlock &newBlock,
                         const std::vector<DatanodeInfo> &newNodes,
-                        const std::vector<std::string> &storageIDs) /* throw (HdfsIOException) */;
+                        const std::vector<std::string> &
+                            storageIDs) /* throw (HdfsIOException) */;
 
-    //Idempotent
+    // Idempotent
     Token getDelegationToken(const std::string &renewer)
-    /* throws IOException*/;
+        /* throws IOException*/;
 
-    //Idempotent
+    // Idempotent
     int64_t renewDelegationToken(const Token &token)
-    /*throws IOException*/;
+        /*throws IOException*/;
 
-    //Idempotent
+    // Idempotent
     void cancelDelegationToken(const Token &token)
-    /*throws IOException*/;
+        /*throws IOException*/;
 
 private:
     void invoke(const RpcCall &call);
+    NamenodeImpl(const NamenodeImpl &other);
+    NamenodeImpl &operator=(const NamenodeImpl &other);
 
-private:
     RpcAuth auth;
     RpcClient &client;
     RpcConfig conf;
     RpcProtocolInfo protocol;
     RpcServerInfo server;
 };
-
 }
 }
 

+ 31 - 19
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc

@@ -16,9 +16,11 @@
  * limitations under the License.
  */
 
+#include "Config.h"
+#include "ConfigImpl.h"
 #include "NamenodeInfo.h"
+#include "StatusInternal.h"
 #include "StringUtil.h"
-#include "XmlConfig.h"
 
 #include <string>
 #include <vector>
@@ -35,25 +37,35 @@ const char *const DFS_NAMENODE_HA = "dfs.ha.namenodes";
 const char *const DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
 const char *const DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
 
-std::vector<NamenodeInfo> NamenodeInfo::GetHANamenodeInfo(
-    const std::string & service, const Config & conf) {
-    std::vector<NamenodeInfo> retval;
-    std::string strNameNodes = StringTrim(
-              conf.getString(std::string(DFS_NAMENODE_HA) + "." + service));
-    std::vector<std::string> nns = StringSplit(strNameNodes, ",");
-    retval.resize(nns.size());
-
-    for (size_t i = 0; i < nns.size(); ++i) {
-        std::string dfsRpcAddress = StringTrim(
-              std::string(DFS_NAMENODE_RPC_ADDRESS_KEY) + "." + service + "."
-              + StringTrim(nns[i]));
-        std::string dfsHttpAddress = 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, "")));
+Status NamenodeInfo::GetHANamenodeInfo(const std::string &service,
+                                       const Config &c,
+                                       std::vector<NamenodeInfo> *output) {
+    ConfigImpl &conf = *c.impl;
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
+
+    try {
+        std::vector<NamenodeInfo> &retval = *output;
+        std::string strNameNodes = StringTrim(
+            conf.getString(std::string(DFS_NAMENODE_HA) + "." + service));
+        std::vector<std::string> nns = StringSplit(strNameNodes, ",");
+        retval.resize(nns.size());
+
+        for (size_t i = 0; i < nns.size(); ++i) {
+            std::string dfsRpcAddress =
+                StringTrim(std::string(DFS_NAMENODE_RPC_ADDRESS_KEY) + "." +
+                           service + "." + StringTrim(nns[i]));
+            std::string dfsHttpAddress =
+                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, "")));
+        }
+
+    } catch (...) {
+        return CreateStatusFromException(current_exception());
     }
 
-    return retval;
+    return Status::OK();
 }
 }

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h

@@ -19,7 +19,8 @@
 #ifndef _HDFS_LIBHDFS_SERVER_NAMENODEINFO_H_
 #define _HDFS_LIBHDFS_SERVER_NAMENODEINFO_H_
 
-#include "XmlConfig.h"
+#include "Config.h"
+#include "Status.h"
 
 #include <string>
 #include <vector>
@@ -46,14 +47,14 @@ public:
         rpc_addr = rpcAddr;
     }
 
-    static std::vector<NamenodeInfo> GetHANamenodeInfo(
-          const std::string &service, const Config &conf);
+    static Status GetHANamenodeInfo(const std::string &service,
+                                    const Config &conf,
+                                    std::vector<NamenodeInfo> *output);
 
 private:
     std::string rpc_addr;
     std::string http_addr;
 };
-
 }
 
 #endif /* _HDFS_LIBHDFS_SERVER_NAMENODEINFO_H_ */

+ 36 - 32
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h

@@ -24,51 +24,54 @@
 #include "SharedPtr.h"
 #include "Thread.h"
 
+#include <string>
+#include <vector>
+
 namespace hdfs {
 namespace internal {
 
-class NamenodeProxy: public Namenode {
+class NamenodeProxy : public Namenode {
 public:
     NamenodeProxy(const std::vector<NamenodeInfo> &namenodeInfos,
-            const std::string &tokenService,
-            const SessionConfig &c, const RpcAuth &a);
+                  const std::string &tokenService, const SessionConfig &c,
+                  const RpcAuth &a);
     ~NamenodeProxy();
 
 public:
-
     void getBlockLocations(const std::string &src, int64_t offset,
-            int64_t length, LocatedBlocks &lbs);
+                           int64_t length, LocatedBlocks &lbs);
 
     void create(const std::string &src, const Permission &masked,
-            const std::string &clientName, int flag, bool createParent,
-            short replication, int64_t blockSize);
+                const std::string &clientName, int flag, bool createParent,
+                short replication, int64_t blockSize);
 
     shared_ptr<LocatedBlock> append(const std::string &src,
-            const std::string &clientName);
+                                    const std::string &clientName);
 
     bool setReplication(const std::string &src, short replication);
 
     void setPermission(const std::string &src, const Permission &permission);
 
     void setOwner(const std::string &src, const std::string &username,
-            const std::string &groupname);
+                  const std::string &groupname);
 
     void abandonBlock(const ExtendedBlock &b, const std::string &src,
-            const std::string &holder);
+                      const std::string &holder);
 
-    shared_ptr<LocatedBlock> addBlock(const std::string &src,
-            const std::string &clientName, const ExtendedBlock *previous,
-            const std::vector<DatanodeInfo> &excludeNodes);
+    shared_ptr<LocatedBlock> addBlock(
+        const std::string &src, const std::string &clientName,
+        const ExtendedBlock *previous,
+        const std::vector<DatanodeInfo> &excludeNodes);
 
-    shared_ptr<LocatedBlock> getAdditionalDatanode(const std::string &src,
-            const ExtendedBlock &blk,
-            const std::vector<DatanodeInfo> &existings,
-            const std::vector<std::string> &storageIDs,
-            const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
-            const std::string &clientName);
+    shared_ptr<LocatedBlock> getAdditionalDatanode(
+        const std::string &src, const ExtendedBlock &blk,
+        const std::vector<DatanodeInfo> &existings,
+        const std::vector<std::string> &storageIDs,
+        const std::vector<DatanodeInfo> &excludes, int numAdditionalNodes,
+        const std::string &clientName);
 
     bool complete(const std::string &src, const std::string &clientName,
-            const ExtendedBlock *last);
+                  const ExtendedBlock *last);
 
     void reportBadBlocks(const std::vector<LocatedBlock> &blocks);
 
@@ -85,10 +88,10 @@ public:
     bool deleteFile(const std::string &src, bool recursive);
 
     bool mkdirs(const std::string &src, const Permission &masked,
-            bool createParent);
+                bool createParent);
 
     bool getListing(const std::string &src, const std::string &startAfter,
-            bool needLocation, std::vector<FileStatus> &dl);
+                    bool needLocation, std::vector<FileStatus> &dl);
 
     void renewLease(const std::string &clientName);
 
@@ -103,24 +106,25 @@ public:
     FileStatus getFileLinkInfo(const std::string &src);
 
     void setQuota(const std::string &path, int64_t namespaceQuota,
-            int64_t diskspaceQuota);
+                  int64_t diskspaceQuota);
 
     void fsync(const std::string &src, const std::string &client);
 
     void setTimes(const std::string &src, int64_t mtime, int64_t atime);
 
     void createSymlink(const std::string &target, const std::string &link,
-            const Permission &dirPerm, bool createParent);
+                       const Permission &dirPerm, bool createParent);
 
     std::string getLinkTarget(const std::string &path);
 
-    shared_ptr<LocatedBlock> updateBlockForPipeline(const ExtendedBlock &block,
-            const std::string &clientName);
+    shared_ptr<LocatedBlock> updateBlockForPipeline(
+        const ExtendedBlock &block, const std::string &clientName);
 
     void updatePipeline(const std::string &clientName,
-            const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
-            const std::vector<DatanodeInfo> &newNodes,
-            const std::vector<std::string> &storageIDs);
+                        const ExtendedBlock &oldBlock,
+                        const ExtendedBlock &newBlock,
+                        const std::vector<DatanodeInfo> &newNodes,
+                        const std::vector<std::string> &storageIDs);
 
     Token getDelegationToken(const std::string &renewer);
 
@@ -131,18 +135,18 @@ public:
     void close();
 
 private:
+    NamenodeProxy(const NamenodeProxy &other);
+    NamenodeProxy &operator=(const NamenodeProxy &other);
     shared_ptr<Namenode> getActiveNamenode(uint32_t &oldValue);
     void failoverToNextNamenode(uint32_t oldValue);
 
-private:
     bool enableNamenodeHA;
     int maxNamenodeHARetry;
     mutex mut;
     std::string clusterid;
-    std::vector<shared_ptr<Namenode> > namenodes;
+    std::vector<shared_ptr<Namenode>> namenodes;
     uint32_t currentNamenode;
 };
-
 }
 }