HDFS-7012. Add hdfs native client RPC functionality (Zhanwei Wang via Colin P. McCabe)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Platform.cmake b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Platform.cmake
index 296734b..f5f18e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Platform.cmake
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Platform.cmake
@@ -28,8 +28,6 @@
     EXECUTE_PROCESS(COMMAND ${CMAKE_CXX_COMPILER} --version  OUTPUT_VARIABLE COMPILER_OUTPUT)
     
     STRING(REGEX MATCH "[^0-9]*([0-9]\\.[0-9]\\.[0-9])" GCC_COMPILER_VERSION ${COMPILER_OUTPUT})
-    MESSAGE(STATUS "WATERMELON MATCHALL "[0-9]" GCC_COMPILER_VERSION ${GCC_COMPILER_VERSION}")
-    MESSAGE(STATUS "WATERMELON COMPILER_OUTPUT=${COMPILER_OUTPUT}")
     STRING(REGEX MATCHALL "[0-9]" GCC_COMPILER_VERSION ${GCC_COMPILER_VERSION})
     
     LIST(LENGTH GCC_COMPILER_VERSION GCC_COMPILER_VERSION_LEN)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/bootstrap b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/bootstrap
old mode 100644
new mode 100755
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
index 9c0145a..5ca3716 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
@@ -34,23 +34,24 @@
 CONFIGURE_FILE(doxyfile.in doxyfile)
 
 AUTO_SOURCES(files "*.cc" "RECURSE" "${CMAKE_CURRENT_SOURCE_DIR}")
-LIST(APPEND libhdfs3_SOURCES ${files})
+LIST(APPEND LIBHDFS3_SOURCES ${files})
 
 AUTO_SOURCES(files "*.c" "RECURSE" "${CMAKE_CURRENT_SOURCE_DIR}")
-LIST(APPEND libhdfs3_SOURCES ${files})
+LIST(APPEND LIBHDFS3_SOURCES ${files})
 
 AUTO_SOURCES(files "*.h" "RECURSE" "${CMAKE_CURRENT_SOURCE_DIR}")
-LIST(APPEND libhdfs3_SOURCES ${files})
+LIST(APPEND LIBHDFS3_SOURCES ${files})
 
 AUTO_SOURCES(libhdfs3_PROTO_FILES "proto/*.proto" "RECURSE" "${CMAKE_CURRENT_SOURCE_DIR}")
 SET(libhdfs3_PROTO_FILES ${libhdfs3_PROTO_FILES} PARENT_SCOPE)
 
-#PROTOBUF_GENERATE_CPP(libhdfs3_PROTO_SOURCES libhdfs3_PROTO_HEADERS ${libhdfs3_PROTO_FILES})
+INCLUDE(GenerateProtobufs.cmake)
+INCLUDE_DIRECTORIES("${CMAKE_BINARY_DIR}")
 
 SET(HEADER client/hdfs.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})
+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})
 
 ADD_CUSTOM_COMMAND(
 	TARGET libhdfs3-shared libhdfs3-static
@@ -122,7 +123,7 @@
         ARCHIVE DESTINATION lib)
 INSTALL(FILES ${HEADER} DESTINATION include/hdfs)
 
-SET(libhdfs3_SOURCES ${libhdfs3_SOURCES} PARENT_SCOPE)
+SET(LIBHDFS3_SOURCES ${LIBHDFS3_SOURCES} PARENT_SCOPE)
 SET(libhdfs3_PLATFORM_HEADER_DIR ${CMAKE_CURRENT_BINARY_DIR} PARENT_SCOPE)
 SET(libhdfs3_ROOT_SOURCES_DIR ${libhdfs3_ROOT_SOURCES_DIR} PARENT_SCOPE)
 SET(libhdfs3_COMMON_SOURCES_DIR ${libhdfs3_COMMON_SOURCES_DIR} PARENT_SCOPE)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
new file mode 100644
index 0000000..50efc2c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
@@ -0,0 +1,66 @@
+#
+# 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.
+#
+
+MESSAGE(STATUS "Processing hadoop protobuf definitions.")
+
+function(COPY_IF_CHANGED TARGET)
+    file(MAKE_DIRECTORY "${TARGET}")
+    foreach(PB_PATH ${ARGN})
+        get_filename_component(PB_FILENAME "${PB_PATH}" NAME)
+        configure_file("${PB_PATH}" "${TARGET}/${PB_FILENAME}" COPY_ONLY)
+    endforeach()
+endfunction(COPY_IF_CHANGED TARGET)
+
+get_filename_component(R "${PROJECT_SOURCE_DIR}/../../../../../" REALPATH)
+
+COPY_IF_CHANGED("${CMAKE_BINARY_DIR}/common_pb"
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/IpcConnectionContext.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshCallQueueProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/Security.proto
+    ${R}/hadoop-common-project/hadoop-common/src/main/proto/ZKFCProtocol.proto
+)
+
+COPY_IF_CHANGED("${CMAKE_BINARY_DIR}/hdfs_pb"
+    #${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HAZKInfo.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
+    #${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
+    ${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
+)
+
+AUTO_SOURCES(PB_SOURCES "*.proto" "RECURSE" "${CMAKE_BINARY_DIR}")
+MESSAGE("PB_SOURCES = ${PB_SOURCES}")
+
+PROTOBUF_GENERATE_CPP(LIBHDFS3_PROTO_SOURCES LIBHDFS3_PROTO_HEADERS "${PB_SOURCES}")
+set(${LIBHDFS3_PROTO_SOURCES} ${LIBHDFS3_PROTO_HEADERS} PARENT_SCOPE)
+MESSAGE("LIBHDFS3_PROTO_SOURCES = ${LIBHDFS3_PROTO_SOURCES}")
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h
new file mode 100644
index 0000000..fe583cf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h
@@ -0,0 +1,57 @@
+/**
+ * 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_BLOCKREADER_H_
+#define _HDFS_LIBHDFS3_CLIENT_BLOCKREADER_H_
+
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+class BlockReader {
+public:
+    virtual ~BlockReader() {
+    }
+
+    /**
+     * Get how many bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking.
+     */
+    virtual int64_t available() = 0;
+
+    /**
+     * To read data from block.
+     * @param buf the buffer used to filled.
+     * @param size the number of bytes to be read.
+     * @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;
+
+    /**
+     * Move the cursor forward len bytes.
+     * @param len The number of bytes to skip.
+     */
+    virtual void skip(int64_t len) = 0;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_BLOCKREADER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocol.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocol.h
new file mode 100644
index 0000000..73bc880
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocol.h
@@ -0,0 +1,101 @@
+/**
+ * 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_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOL_H_
+#define _HDFS_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOL_H_
+
+#include "client/Token.h"
+#include "server/DatanodeInfo.h"
+#include "server/ExtendedBlock.h"
+
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * Transfer data to/from datanode using a streaming protocol.
+ */
+class DataTransferProtocol {
+public:
+    virtual ~DataTransferProtocol() {
+    }
+    /**
+     * Read a block.
+     *
+     * @param blk the block being read.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param blockOffset offset of the block.
+     * @param length maximum number of bytes for this read.
+     */
+    virtual void readBlock(const ExtendedBlock & blk,
+                           const Token & blockToken, const char * clientName,
+                           int64_t blockOffset, int64_t length) = 0;
+
+    /**
+     * Write a block to a datanode pipeline.
+     *
+     * @param blk the block being written.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param targets target datanodes in the pipeline.
+     * @param source source datanode.
+     * @param stage pipeline stage.
+     * @param pipelineSize the size of the pipeline.
+     * @param minBytesRcvd minimum number of bytes received.
+     * @param maxBytesRcvd maximum number of bytes received.
+     * @param latestGenerationStamp the latest generation stamp of the block.
+     */
+    virtual void writeBlock(const ExtendedBlock & blk,
+                            const Token & blockToken, const char * clientName,
+                            const std::vector<DatanodeInfo> & targets, int stage,
+                            int pipelineSize, int64_t minBytesRcvd, int64_t maxBytesRcvd,
+                            int64_t latestGenerationStamp, int checksumType,
+                            int bytesPerChecksum) = 0;
+
+    /**
+     * Transfer a block to another datanode.
+     * The block stage must be
+     * either {@link BlockConstructionStage#TRANSFER_RBW}
+     * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
+     *
+     * @param blk the block being transferred.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param targets target datanodes.
+     */
+    virtual void transferBlock(const ExtendedBlock & blk,
+                               const Token & blockToken, const char * clientName,
+                               const std::vector<DatanodeInfo> & targets) = 0;
+
+    /**
+     * Get block checksum (MD5 of CRC32).
+     *
+     * @param blk a block.
+     * @param blockToken security token for accessing the block.
+     * @throw HdfsIOException
+     */
+    virtual void blockChecksum(const ExtendedBlock & blk,
+                               const Token & blockToken) = 0;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.cc
new file mode 100644
index 0000000..a51d1dd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.cc
@@ -0,0 +1,179 @@
+/**
+ * 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 "client/Token.h"
+#include "datatransfer.pb.h"
+#include "DataTransferProtocolSender.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "hdfs.pb.h"
+#include "Security.pb.h"
+#include "WriteBuffer.h"
+
+using namespace google::protobuf;
+
+using namespace hadoop::hdfs;
+using namespace hadoop::common;
+
+namespace hdfs {
+namespace internal {
+
+static inline void Send(Socket &sock, DataTransferOp op, Message * msg,
+              int writeTimeout) {
+    WriteBuffer buffer;
+    buffer.writeBigEndian(static_cast<int16_t>(DATA_TRANSFER_VERSION));
+    buffer.write(static_cast<char>(op));
+    int msgSize = msg->ByteSize();
+    buffer.writeVarint32(msgSize);
+    char * b = buffer.alloc(msgSize);
+
+    if (!msg->SerializeToArray(b, msgSize)) {
+        THROW(HdfsIOException,
+              "DataTransferProtocolSender cannot serialize header to "
+              "send buffer.");
+    }
+
+    sock.writeFully(buffer.getBuffer(0), buffer.getDataSize(0), writeTimeout);
+}
+
+static inline void BuildBaseHeader(const ExtendedBlock &block,
+              const Token &accessToken, BaseHeaderProto * header) {
+    ExtendedBlockProto * eb = header->mutable_block();
+    TokenProto * token = header->mutable_token();
+    eb->set_blockid(block.getBlockId());
+    eb->set_generationstamp(block.getGenerationStamp());
+    eb->set_numbytes(block.getNumBytes());
+    eb->set_poolid(block.getPoolId());
+    token->set_identifier(accessToken.getIdentifier());
+    token->set_password(accessToken.getPassword());
+    token->set_kind(accessToken.getKind());
+    token->set_service(accessToken.getService());
+}
+
+static inline void BuildClientHeader(const ExtendedBlock &block,
+                                     const Token &accessToken, const char * clientName,
+                                     ClientOperationHeaderProto * header) {
+    header->set_clientname(clientName);
+    BuildBaseHeader(block, accessToken, header->mutable_baseheader());
+}
+
+static inline void BuildNodeInfo(const DatanodeInfo &node,
+                                 DatanodeInfoProto * info) {
+    DatanodeIDProto * id = info->mutable_id();
+    id->set_hostname(node.getHostName());
+    id->set_infoport(node.getInfoPort());
+    id->set_ipaddr(node.getIpAddr());
+    id->set_ipcport(node.getIpcPort());
+    id->set_datanodeuuid(node.getDatanodeId());
+    id->set_xferport(node.getXferPort());
+    info->set_location(node.getLocation());
+}
+
+static inline void BuildNodesInfo(const std::vector<DatanodeInfo> &nodes,
+                                  RepeatedPtrField<DatanodeInfoProto> * infos) {
+    for (std::size_t i = 0; i < nodes.size(); ++i) {
+        BuildNodeInfo(nodes[i], infos->Add());
+    }
+}
+
+DataTransferProtocolSender::DataTransferProtocolSender(Socket &sock,
+        int writeTimeout, const std::string &datanodeAddr) :
+    sock(sock), writeTimeout(writeTimeout), datanode(datanodeAddr) {
+}
+
+DataTransferProtocolSender::~DataTransferProtocolSender() {
+}
+
+void DataTransferProtocolSender::readBlock(const ExtendedBlock &blk,
+        const Token &blockToken, const char * clientName,
+        int64_t blockOffset, int64_t length) {
+    try {
+        OpReadBlockProto op;
+        op.set_len(length);
+        op.set_offset(blockOffset);
+        BuildClientHeader(blk, blockToken, clientName, op.mutable_header());
+        Send(sock, READ_BLOCK, &op, writeTimeout);
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "DataTransferProtocolSender cannot send read request "
+                     "to datanode %s.", datanode.c_str());
+    }
+}
+
+void DataTransferProtocolSender::writeBlock(const ExtendedBlock &blk,
+        const Token &blockToken, const char * clientName,
+        const std::vector<DatanodeInfo> &targets, int stage, int pipelineSize,
+        int64_t minBytesRcvd, int64_t maxBytesRcvd,
+        int64_t latestGenerationStamp, int checksumType, int bytesPerChecksum) {
+    try {
+        OpWriteBlockProto op;
+        op.set_latestgenerationstamp(latestGenerationStamp);
+        op.set_minbytesrcvd(minBytesRcvd);
+        op.set_maxbytesrcvd(maxBytesRcvd);
+        op.set_pipelinesize(targets.size());
+        op.set_stage((OpWriteBlockProto_BlockConstructionStage) stage);
+        BuildClientHeader(blk, blockToken, clientName, op.mutable_header());
+        ChecksumProto * ck = op.mutable_requestedchecksum();
+        ck->set_bytesperchecksum(bytesPerChecksum);
+        ck->set_type((ChecksumTypeProto) checksumType);
+        BuildNodesInfo(targets, op.mutable_targets());
+        Send(sock, WRITE_BLOCK, &op, writeTimeout);
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "DataTransferProtocolSender cannot send write request "
+                     "to datanode %s.", datanode.c_str());
+    }
+}
+
+void DataTransferProtocolSender::transferBlock(const ExtendedBlock &blk,
+        const Token &blockToken, const char * clientName,
+        const std::vector<DatanodeInfo> &targets) {
+    try {
+        OpTransferBlockProto op;
+        BuildClientHeader(blk, blockToken, clientName, op.mutable_header());
+        BuildNodesInfo(targets, op.mutable_targets());
+        Send(sock, TRANSFER_BLOCK, &op, writeTimeout);
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "DataTransferProtocolSender cannot send transfer "
+                     "request to datanode %s.", datanode.c_str());
+    }
+}
+
+void DataTransferProtocolSender::blockChecksum(const ExtendedBlock &blk,
+        const Token &blockToken) {
+    try {
+        //TODO
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "DataTransferProtocolSender cannot send checksum "
+                     "request to datanode %s.", datanode.c_str());
+    }
+}
+
+}
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.h
new file mode 100644
index 0000000..c7cbd38
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.h
@@ -0,0 +1,120 @@
+/**
+ * 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_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOLSENDER_H_
+#define _HDFS_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOLSENDER_H_
+
+#include "DataTransferProtocol.h"
+#include "network/Socket.h"
+
+/**
+ * Version 28:
+ *    Declare methods in DataTransferProtocol interface.
+ */
+#define DATA_TRANSFER_VERSION 28
+
+namespace hdfs {
+namespace internal {
+
+enum DataTransferOp {
+    WRITE_BLOCK = 80,
+    READ_BLOCK = 81,
+    READ_METADATA = 82,
+    REPLACE_BLOCK = 83,
+    COPY_BLOCK = 84,
+    BLOCK_CHECKSUM = 85,
+    TRANSFER_BLOCK = 86
+};
+
+/**
+ * Transfer data to/from datanode using a streaming protocol.
+ */
+class DataTransferProtocolSender: public DataTransferProtocol {
+public:
+    DataTransferProtocolSender(Socket & sock, int writeTimeout,
+                               const std::string & datanodeAddr);
+
+    virtual ~DataTransferProtocolSender();
+
+    /**
+     * Read a block.
+     *
+     * @param blk the block being read.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param blockOffset offset of the block.
+     * @param length maximum number of bytes for this read.
+     */
+    virtual void readBlock(const ExtendedBlock & blk, const Token & blockToken,
+                           const char * clientName, int64_t blockOffset, int64_t length);
+
+    /**
+     * Write a block to a datanode pipeline.
+     *
+     * @param blk the block being written.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param targets target datanodes in the pipeline.
+     * @param source source datanode.
+     * @param stage pipeline stage.
+     * @param pipelineSize the size of the pipeline.
+     * @param minBytesRcvd minimum number of bytes received.
+     * @param maxBytesRcvd maximum number of bytes received.
+     * @param latestGenerationStamp the latest generation stamp of the block.
+     */
+    virtual void writeBlock(const ExtendedBlock & blk, const Token & blockToken,
+                            const char * clientName, const std::vector<DatanodeInfo> & targets,
+                            int stage, int pipelineSize, int64_t minBytesRcvd,
+                            int64_t maxBytesRcvd, int64_t latestGenerationStamp,
+                            int checksumType, int bytesPerChecksum);
+
+    /**
+     * Transfer a block to another datanode.
+     * The block stage must be
+     * either {@link BlockConstructionStage#TRANSFER_RBW}
+     * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
+     *
+     * @param blk the block being transferred.
+     * @param blockToken security token for accessing the block.
+     * @param clientName client's name.
+     * @param targets target datanodes.
+     */
+    virtual void transferBlock(const ExtendedBlock & blk,
+                               const Token & blockToken, const char * clientName,
+                               const std::vector<DatanodeInfo> & targets);
+
+    /**
+     * Get block checksum (MD5 of CRC32).
+     *
+     * @param blk a block.
+     * @param blockToken security token for accessing the block.
+     * @throw HdfsIOException
+     */
+    virtual void blockChecksum(const ExtendedBlock & blk,
+                               const Token & blockToken);
+
+private:
+    Socket & sock;
+    int writeTimeout;
+    std::string datanode;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS_3_CLIENT_DATATRANSFERPROTOCOLSENDER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileStatus.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileStatus.h
new file mode 100644
index 0000000..ff4de35
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileStatus.h
@@ -0,0 +1,159 @@
+/**
+ * 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_FILESTATUS_H_
+#define _HDFS_LIBHDFS3_CLIENT_FILESTATUS_H_
+
+#include "Permission.h"
+
+#include <stdint.h>
+#include <string>
+
+namespace hdfs {
+
+class FileStatus {
+public:
+    FileStatus() :
+        isdir(false), atime(0), blocksize(0), length(0), mtime(
+            0), permission(0644), replications(0) {
+    }
+
+    int64_t getAccessTime() const {
+        return atime;
+    }
+
+    void setAccessTime(int64_t accessTime) {
+        atime = accessTime;
+    }
+
+    short getReplication() const {
+        return replications;
+    }
+
+    void setReplication(short blockReplication) {
+        replications = blockReplication;
+    }
+
+    int64_t getBlockSize() const {
+        return blocksize;
+    }
+
+    void setBlocksize(int64_t blocksize) {
+        this->blocksize = blocksize;
+    }
+
+    const char *getGroup() const {
+        return group.c_str();
+    }
+
+    void setGroup(const char * group) {
+        this->group = group;
+    }
+
+    /**
+     * Is this a directory?
+     * @return true if this is a directory
+     */
+    bool isDirectory() const {
+        return isdir;
+    }
+
+    void setIsdir(bool isdir) {
+        this->isdir = isdir;
+    }
+
+    int64_t getLength() const {
+        return length;
+    }
+
+    void setLength(int64_t length) {
+        this->length = length;
+    }
+
+    int64_t getModificationTime() const {
+        return mtime;
+    }
+
+    void setModificationTime(int64_t modificationTime) {
+        mtime = modificationTime;
+    }
+
+    const char *getOwner() const {
+        return owner.c_str();
+    }
+
+    void setOwner(const char * owner) {
+        this->owner = owner;
+    }
+
+    const char *getPath() const {
+        return path.c_str();
+    }
+
+    void setPath(const char * path) {
+        this->path = path;
+    }
+
+    const Permission &getPermission() const {
+        return permission;
+    }
+
+    void setPermission(const Permission & permission) {
+        this->permission = permission;
+    }
+
+    const char *getSymlink() const {
+        return symlink.c_str();
+    }
+
+    void setSymlink(const char *symlink) {
+        this->symlink = symlink;
+    }
+
+    /**
+     * Is this a file?
+     * @return true if this is a file
+     */
+    bool isFile() {
+        return !isdir && !isSymlink();
+    }
+
+    /**
+     * Is this a symbolic link?
+     * @return true if this is a symbolic link
+     */
+    bool isSymlink() {
+        return !symlink.empty();
+    }
+
+private:
+    bool isdir;
+    int64_t atime;
+    int64_t blocksize;
+    int64_t length;
+    int64_t mtime;
+    Permission permission;
+    short replications;
+    std::string group;
+    std::string owner;
+    std::string path;
+    std::string symlink;
+};
+
+}
+
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h
new file mode 100644
index 0000000..ddd9434
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h
@@ -0,0 +1,90 @@
+/**
+ * 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_INPUTSTREAM_H_
+#define _HDFS_LIBHDFS3_CLIENT_INPUTSTREAM_H_
+
+#include "FileSystem.h"
+
+namespace hdfs {
+namespace internal {
+class InputStreamInter;
+}
+
+/**
+ * A input stream used read data from hdfs.
+ */
+class InputStream {
+public:
+    InputStream();
+
+    ~InputStream();
+
+    /**
+     * Open a file to read
+     * @param fs hdfs file system.
+     * @param path the file to be read.
+     * @param verifyChecksum verify the checksum.
+     */
+    void open(FileSystem & fs, const char * 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.
+     */
+    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);
+
+    /**
+     * Get how many bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking.
+     */
+    int64_t available();
+
+    /**
+     * To move the file point to the given position.
+     * @param pos the given position.
+     */
+    void seek(int64_t pos);
+
+    /**
+     * To get the current file point position.
+     * @return the position of current file point.
+     */
+    int64_t tell();
+
+    /**
+     * Close the stream.
+     */
+    void close();
+
+private:
+    Internal::InputStreamInter * impl;
+};
+
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_INPUTSTREAM_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h
new file mode 100644
index 0000000..8723344
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#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"
+#include "Thread.h"
+#include "UnorderedMap.h"
+#include "rpc/RpcAuth.h"
+#include "server/BlockLocalPathInfo.h"
+#include "server/Datanode.h"
+#include "server/LocatedBlock.h"
+#include "server/LocatedBlocks.h"
+
+#ifdef MOCK
+#include "TestDatanodeStub.h"
+#endif
+
+namespace hdfs {
+namespace internal {
+
+typedef std::pair<int64_t, std::string> LocalBlockInforCacheKey;
+typedef LruMap<LocalBlockInforCacheKey, BlockLocalPathInfo> LocalBlockInforCacheType;
+
+/**
+ * A input stream used read data from hdfs.
+ */
+class InputStreamImpl: public InputStreamInter {
+public:
+    InputStreamImpl();
+    ~InputStreamImpl();
+
+    /**
+     * Open a file to read
+     * @param fs hdfs file system.
+     * @param path the file to be read.
+     * @param verifyChecksum verify the checksum.
+     */
+    void open(shared_ptr<FileSystemInter> 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.
+     */
+    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);
+
+    int64_t available();
+
+    /**
+     * To move the file point to the given position.
+     * @param pos the given position.
+     */
+    void seek(int64_t pos);
+
+    /**
+     * To get the current file point position.
+     * @return the position of current file point.
+     */
+    int64_t tell();
+
+    /**
+     * Close the stream.
+     */
+    void close();
+
+    std::string toString();
+
+private:
+    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);
+    int64_t getFileLength();
+    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,
+                      bool verifyChecksum);
+    void readFullyInternal(char * buf, int64_t size);
+    void seekInternal(int64_t pos);
+    void seekToBlock(const LocatedBlock & lb);
+    void setupBlockReader(bool temporaryDisableLocalRead);
+    void updateBlockInfos();
+
+private:
+    bool closed;
+    bool localRead;
+    bool readFromUnderConstructedBlock;
+    bool verify;
+    DatanodeInfo curNode;
+    exception_ptr lastError;
+    FileStatus fileInfo;
+    int maxGetBlockInfoRetry;
+    int64_t cursor;
+    int64_t endOfCurBlock;
+    int64_t lastBlockBeingWrittenLength;
+    int64_t prefetchSize;
+    RpcAuth auth;
+    shared_ptr<BlockReader> blockReader;
+    shared_ptr<FileSystemInter> filesystem;
+    shared_ptr<LocatedBlock> curBlock;
+    shared_ptr<LocatedBlocks> lbs;
+    shared_ptr<SessionConfig> conf;
+    std::string path;
+    std::vector<DatanodeInfo> failedNodes;
+    std::vector<char> localReaderBuffer;
+
+    static mutex MutLocalBlockInforCache;
+    static unordered_map<uint32_t, shared_ptr<LocalBlockInforCacheType> > LocalBlockInforCache;
+#ifdef MOCK
+private:
+    hdfs::mock::TestDatanodeStub * stub;
+#endif
+};
+
+}
+}
+
+#ifdef NEED_BOOST
+
+namespace boost {
+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]));
+    }
+};
+}
+
+#else
+
+namespace std {
+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]));
+    }
+};
+}
+
+#endif
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_INPUTSTREAMIMPL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamInter.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamInter.h
new file mode 100644
index 0000000..c0da813
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamInter.h
@@ -0,0 +1,96 @@
+/**
+ * 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_INPUTSTREAMINTER_H_
+#define _HDFS_LIBHDFS3_CLIENT_INPUTSTREAMINTER_H_
+
+#include <SharedPtr.h>
+
+#include <stdint.h>
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class FileSystemInter;
+
+/**
+ * A input stream used read data from hdfs.
+ */
+class InputStreamInter {
+public:
+
+    virtual ~InputStreamInter() {
+    }
+
+    /**
+     * Open a file to read
+     * @param fs hdfs file system.
+     * @param path the file to be read.
+     * @param verifyChecksum verify the checksum.
+     */
+    virtual void open(shared_ptr<FileSystemInter> fs, const char * path,
+                      bool verifyChecksum) = 0;
+
+    /**
+     * 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.
+     */
+    virtual int32_t read(char * buf, int32_t size) = 0;
+
+    /**
+     * 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.
+     */
+    virtual void readFully(char * buf, int64_t size) = 0;
+
+    /**
+     * Get how many bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking.
+     */
+    virtual int64_t available() = 0;
+
+    /**
+     * To move the file point to the given position.
+     * @param pos the given position.
+     */
+    virtual void seek(int64_t pos) = 0;
+
+    /**
+     * To get the current file point position.
+     * @return the position of current file point.
+     */
+    virtual int64_t tell() = 0;
+
+    /**
+     * Close the stream.
+     */
+    virtual void close() = 0;
+
+    /**
+     * Output a readable string of this input stream.
+     */
+    virtual std::string toString() = 0;
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS3_CLIENT_INPUTSTREAMINTER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.cc
new file mode 100644
index 0000000..877758c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.cc
@@ -0,0 +1,113 @@
+/**
+ * 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 "KerberosName.h"
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+#include <regex.h>
+#include <string.h>
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+static void HandleRegError(int rc, regex_t *comp) {
+    std::vector<char> buffer;
+    size_t size = regerror(rc, comp, NULL, 0);
+    buffer.resize(size + 1);
+    regerror(rc, comp, &buffer[0], buffer.size());
+    THROW(HdfsIOException,
+        "KerberosName: Failed to parse Kerberos principal.");
+}
+
+KerberosName::KerberosName() {
+}
+
+KerberosName::KerberosName(const std::string &principal) {
+    parse(principal);
+}
+
+void KerberosName::parse(const std::string &principal) {
+    int rc;
+    static const char * pattern = "([^/@]*)(/([^/@]*))?@([^/@]*)";
+    regex_t comp;
+    regmatch_t pmatch[5];
+
+    if (principal.empty()) {
+        return;
+    }
+
+    memset(&comp, 0, sizeof(regex_t));
+    rc = regcomp(&comp, pattern, REG_EXTENDED);
+
+    if (rc) {
+        HandleRegError(rc, &comp);
+    }
+
+    try {
+        memset(pmatch, 0, sizeof(pmatch));
+        rc = regexec(&comp, principal.c_str(),
+                     sizeof(pmatch) / sizeof(pmatch[1]), pmatch, 0);
+
+        if (rc && rc != REG_NOMATCH) {
+            HandleRegError(rc, &comp);
+        }
+
+        if (rc == REG_NOMATCH) {
+            if (principal.find('@') != principal.npos) {
+                THROW(HdfsIOException,
+                      "KerberosName: Malformed Kerberos name: %s",
+                      principal.c_str());
+            } else {
+                name = principal;
+            }
+        } else {
+            if (pmatch[1].rm_so != -1) {
+                name = principal.substr(pmatch[1].rm_so,
+                                        pmatch[1].rm_eo - pmatch[1].rm_so);
+            }
+
+            if (pmatch[3].rm_so != -1) {
+                host = principal.substr(pmatch[3].rm_so,
+                                        pmatch[3].rm_eo - pmatch[3].rm_so);
+            }
+
+            if (pmatch[4].rm_so != -1) {
+                realm = principal.substr(pmatch[4].rm_so,
+                                         pmatch[4].rm_eo - pmatch[4].rm_so);
+            }
+        }
+    } catch (...) {
+        regfree(&comp);
+        throw;
+    }
+
+    regfree(&comp);
+}
+
+size_t KerberosName::hash_value() const {
+    size_t values[] = { StringHasher(name), StringHasher(host), StringHasher(
+                            realm)
+                      };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.h
new file mode 100644
index 0000000..07e7fb2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.h
@@ -0,0 +1,93 @@
+/**
+ * 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_KERBEROSNAME_H_
+#define _HDFS_LIBHDFS3_CLIENT_KERBEROSNAME_H_
+
+#include <string>
+#include <sstream>
+
+#include "Hash.h"
+
+namespace hdfs {
+namespace internal {
+
+class KerberosName {
+public:
+    KerberosName();
+    KerberosName(const std::string &principal);
+
+    std::string getPrincipal() const {
+        std::stringstream ss;
+        ss << name;
+
+        if (!host.empty()) {
+            ss << "/" << host;
+        }
+
+        if (!realm.empty()) {
+            ss << '@' << realm;
+        }
+
+        return ss.str();
+    }
+
+    const std::string &getHost() const {
+        return host;
+    }
+
+    void setHost(const std::string &host) {
+        this->host = host;
+    }
+
+    const std::string &getName() const {
+        return name;
+    }
+
+    void setName(const std::string &name) {
+        this->name = name;
+    }
+
+    const std::string &getRealm() const {
+        return realm;
+    }
+
+    void setRealm(const std::string &realm) {
+        this->realm = realm;
+    }
+
+    size_t hash_value() const;
+
+    bool operator ==(const KerberosName &other) const {
+        return name == other.name && host == other.host && realm == other.realm;
+    }
+
+private:
+    void parse(const std::string &principal);
+
+private:
+    std::string name;
+    std::string host;
+    std::string realm;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::KerberosName);
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_KERBEROSNAME_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.cc
new file mode 100644
index 0000000..7edcc21
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.cc
@@ -0,0 +1,314 @@
+/**
+ * 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 "BigEndian.h"
+#include "datatransfer.pb.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "HWCrc32c.h"
+#include "LocalBlockReader.h"
+#include "SWCrc32c.h"
+
+#include "hdfs.pb.h"
+
+#include <inttypes.h>
+#include <limits>
+
+#define BMVERSION 1
+#define BMVERSION_SIZE 2
+
+#define HEADER_SIZE (BMVERSION_SIZE + \
+      CHECKSUM_TYPE_SIZE + CHECKSUM_BYTES_PER_CHECKSUM_SIZE)
+
+using hadoop::hdfs::ChecksumTypeProto;
+
+namespace hdfs {
+namespace internal {
+
+LocalBlockReader::LocalBlockReader(const BlockLocalPathInfo &info,
+        const ExtendedBlock &block, int64_t offset, bool verify,
+        SessionConfig &conf, std::vector<char> &buffer) :
+    verify(verify), pbuffer(NULL), pMetaBuffer(NULL), block(block),
+      checksumSize(0), chunkSize(0), position(0), size(0), cursor(
+        0), length(block.getNumBytes()),
+          dataFilePath(info.getLocalBlockPath()), metaFilePath(
+            info.getLocalMetaPath()), buffer(buffer) {
+    exception_ptr lastError;
+
+    try {
+        if (conf.doUseMappedFile()) {
+            metaFd = shared_ptr<MappedFileWrapper>(new MappedFileWrapper);
+            dataFd = shared_ptr<MappedFileWrapper>(new MappedFileWrapper);
+        } else {
+            metaFd = shared_ptr<CFileWrapper>(new CFileWrapper);
+            dataFd = shared_ptr<CFileWrapper>(new CFileWrapper);
+        }
+
+        if (!metaFd->open(metaFilePath)) {
+            THROW(HdfsIOException,
+                  "LocalBlockReader cannot open metadata file \"%s\", %s",
+                  metaFilePath.c_str(), GetSystemErrorInfo(errno));
+        }
+
+        std::vector<char> header;
+        pMetaBuffer = metaFd->read(header, HEADER_SIZE);
+        int16_t version = ReadBigEndian16FromArray(&pMetaBuffer[0]);
+
+        if (BMVERSION != version) {
+            THROW(HdfsIOException,
+                  "LocalBlockReader get an unmatched block, expected block "
+                  "version %d, real version is %d",
+                  BMVERSION, static_cast<int>(version));
+        }
+
+        switch (pMetaBuffer[BMVERSION_SIZE]) {
+        case ChecksumTypeProto::CHECKSUM_NULL:
+            this->verify = false;
+            checksumSize = 0;
+            metaFd.reset();
+            break;
+
+        case ChecksumTypeProto::CHECKSUM_CRC32:
+            THROW(HdfsIOException,
+                  "LocalBlockReader does not support CRC32 checksum.");
+            break;
+
+        case ChecksumTypeProto::CHECKSUM_CRC32C:
+            if (HWCrc32c::available()) {
+                checksum = shared_ptr<Checksum>(new HWCrc32c());
+            } else {
+                checksum = shared_ptr<Checksum>(new SWCrc32c());
+            }
+
+            chunkSize = ReadBigEndian32FromArray(
+                            &pMetaBuffer[BMVERSION_SIZE + CHECKSUM_TYPE_SIZE]);
+            checksumSize = sizeof(int32_t);
+            break;
+
+        default:
+            THROW(HdfsIOException,
+                  "LocalBlockReader cannot recognize checksum type: %d.",
+                  static_cast<int>(pMetaBuffer[BMVERSION_SIZE]));
+        }
+
+        if (verify && chunkSize <= 0) {
+            THROW(HdfsIOException,
+                  "LocalBlockReader get an invalid checksum parameter, "
+                  "bytes per chunk: %d.",
+                  chunkSize);
+        }
+
+        if (!dataFd->open(dataFilePath)) {
+            THROW(HdfsIOException,
+                  "LocalBlockReader cannot open data file \"%s\", %s",
+                  dataFilePath.c_str(), GetSystemErrorInfo(errno));
+        }
+
+        localBufferSize = conf.getLocalReadBufferSize();
+
+        if (verify) {
+            localBufferSize = (localBufferSize + chunkSize - 1) /
+                (chunkSize * chunkSize);
+        }
+
+        if (offset > 0) {
+            skip(offset);
+        }
+    } catch (...) {
+        if (metaFd) {
+            metaFd->close();
+        }
+
+        if (dataFd) {
+            dataFd->close();
+        }
+
+        lastError = current_exception();
+    }
+
+    try {
+        if (lastError != exception_ptr()) {
+            rethrow_exception(lastError);
+        }
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                 "Failed to construct LocalBlockReader for block: %s.",
+                 block.toString().c_str());
+    }
+}
+
+LocalBlockReader::~LocalBlockReader() {
+}
+
+void LocalBlockReader::readAndVerify(int32_t bufferSize) {
+    assert(true == verify);
+    assert(cursor % chunkSize == 0);
+    int chunks = (bufferSize + chunkSize - 1) / chunkSize;
+    pbuffer = dataFd->read(buffer, bufferSize);
+    pMetaBuffer = metaFd->read(metaBuffer, chunks * checksumSize);
+
+    for (int i = 0; i < chunks; ++i) {
+        checksum->reset();
+        int chunk = chunkSize;
+
+        if (chunkSize * (i + 1) > bufferSize) {
+            chunk = bufferSize % chunkSize;
+        }
+
+        checksum->update(&pbuffer[i * chunkSize], chunk);
+        uint32_t target = ReadBigEndian32FromArray(
+                              &pMetaBuffer[i * checksumSize]);
+
+        if (target != checksum->getValue()) {
+            THROW(ChecksumException,
+                  "LocalBlockReader checksum not match for block file: %s",
+                  dataFilePath.c_str());
+        }
+    }
+}
+
+int32_t LocalBlockReader::readInternal(char * buf, int32_t len) {
+    int32_t todo = len;
+
+    /*
+     * read from buffer.
+     */
+    if (position < size) {
+        todo = todo < size - position ? todo : size - position;
+        memcpy(buf, &pbuffer[position], todo);
+        position += todo;
+        cursor += todo;
+        return todo;
+    }
+
+    /*
+     * end of block
+     */
+    todo = todo < length - cursor ? todo : length - cursor;
+
+    if (0 == todo) {
+        return 0;
+    }
+
+    /*
+     * bypass the buffer
+     */
+    if (!verify
+            && (todo > localBufferSize || todo == length - cursor)) {
+        dataFd->copy(buf, todo);
+        cursor += todo;
+        return todo;
+    }
+
+    /*
+     * fill buffer.
+     */
+    int bufferSize = localBufferSize;
+    bufferSize = bufferSize < length - cursor ? bufferSize : length - cursor;
+    assert(bufferSize > 0);
+
+    if (verify) {
+        readAndVerify(bufferSize);
+    } else {
+        pbuffer = dataFd->read(buffer, bufferSize);
+    }
+
+    position = 0;
+    size = bufferSize;
+    assert(position < size);
+    return readInternal(buf, todo);
+}
+
+int32_t LocalBlockReader::read(char *buf, int32_t size) {
+    try {
+        return readInternal(buf, size);
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "LocalBlockReader failed to read from position: %" PRId64
+                     ", length: %d, block: %s.",
+                     cursor, size, block.toString().c_str());
+    }
+
+    assert(!"cannot reach here");
+    return 0;
+}
+
+void LocalBlockReader::skip(int64_t len) {
+    assert(len < length - cursor);
+
+    try {
+        int64_t todo = len;
+
+        while (todo > 0) {
+            /*
+             * skip the data in buffer.
+             */
+            if (size - position > 0) {
+                int batch = todo < size - position ? todo : size - position;
+                position += batch;
+                todo -= batch;
+                cursor += batch;
+                continue;
+            }
+
+            if (verify) {
+                int64_t lastChunkSize = (cursor + todo) % chunkSize;
+                cursor = (cursor + todo) / chunkSize * chunkSize;
+                int64_t metaCursor = HEADER_SIZE
+                                     + checksumSize * (cursor / chunkSize);
+                metaFd->seek(metaCursor);
+                todo = lastChunkSize;
+            } else {
+                cursor += todo;
+                todo = 0;
+            }
+
+            if (cursor > 0) {
+                dataFd->seek(cursor);
+            }
+
+            /*
+             * fill buffer again and verify checksum
+             */
+            if (todo > 0) {
+                assert(true == verify);
+                int bufferSize = localBufferSize;
+                bufferSize =
+                    bufferSize < length - cursor ?
+                    bufferSize : length - cursor;
+                readAndVerify(bufferSize);
+                position = 0;
+                size = bufferSize;
+            }
+        }
+    } catch (const HdfsCanceled &e) {
+        throw;
+    } catch (const HdfsException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "LocalBlockReader failed to skip from position: %" PRId64
+                     ", length: %d, block: %s.",
+                     cursor, size, block.toString().c_str());
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h
new file mode 100644
index 0000000..6118403
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h
@@ -0,0 +1,98 @@
+/**
+ * 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_LOCALBLOCKREADER_H_
+#define _HDFS_LIBHDFS3_CLIENT_LOCALBLOCKREADER_H_
+
+#include "BlockReader.h"
+#include "Checksum.h"
+#include "FileWrapper.h"
+#include "SessionConfig.h"
+#include "common/SharedPtr.h"
+#include "server/BlockLocalPathInfo.h"
+
+#include <stdint.h>
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+class LocalBlockReader: public BlockReader {
+public:
+    LocalBlockReader(const BlockLocalPathInfo & info,
+                     const ExtendedBlock & block, int64_t offset, bool verify,
+                     SessionConfig & conf, std::vector<char> & buffer);
+
+    ~LocalBlockReader();
+
+    /**
+     * Get how many bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking.
+     */
+    virtual int64_t available() {
+        return length - cursor;
+    }
+
+    /**
+     * To read data from block.
+     * @param buf the buffer used to filled.
+     * @param size the number of bytes to be read.
+     * @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);
+
+    /**
+     * Move the cursor forward len bytes.
+     * @param len The number of bytes to skip.
+     */
+    virtual void skip(int64_t len);
+
+private:
+    /**
+     * Fill buffer and verify checksum.
+     * @param bufferSize The size of buffer.
+     */
+    void readAndVerify(int32_t bufferSize);
+    int32_t readInternal(char * buf, int32_t len);
+
+private:
+    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.
+    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> metaBuffer;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_LOCALBLOCKREADER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.cc
new file mode 100644
index 0000000..f46941a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.cc
@@ -0,0 +1,147 @@
+/**
+ * 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 "BigEndian.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "Packet.h"
+#include "PacketHeader.h"
+
+namespace hdfs {
+namespace internal {
+
+Packet::Packet() :
+    lastPacketInBlock(false), syncBlock(false), checksumPos(0), checksumSize(0),
+    checksumStart(0), dataPos(0), dataStart(0), headerStart(0), maxChunks(
+        0), numChunks(0), offsetInBlock(0), seqno(HEART_BEAT_SEQNO) {
+    buffer.resize(PacketHeader::GetPkgHeaderSize());
+}
+
+Packet::Packet(int pktSize, int chunksPerPkt, int64_t offsetInBlock,
+               int64_t seqno, int checksumSize) :
+    lastPacketInBlock(false), syncBlock(false), checksumSize(checksumSize), headerStart(0),
+    maxChunks(chunksPerPkt), numChunks(0), offsetInBlock(offsetInBlock), seqno(seqno), buffer(pktSize) {
+    checksumPos = checksumStart = PacketHeader::GetPkgHeaderSize();
+    dataPos = dataStart = checksumStart + chunksPerPkt * checksumSize;
+    assert(dataPos >= 0);
+}
+
+void Packet::reset(int pktSize, int chunksPerPkt, int64_t offsetInBlock,
+                   int64_t seqno, int checksumSize) {
+    lastPacketInBlock = false;
+    syncBlock = false;
+    this->checksumSize = checksumSize;
+    headerStart = 0;
+    maxChunks = chunksPerPkt;
+    numChunks = 0;
+    this->offsetInBlock = offsetInBlock;
+    this->seqno = seqno;
+    checksumPos = checksumStart = PacketHeader::GetPkgHeaderSize();
+    dataPos = dataStart = checksumStart + chunksPerPkt * checksumSize;
+
+    if (pktSize > static_cast<int>(buffer.size())) {
+        buffer.resize(pktSize);
+    }
+
+    assert(dataPos >= 0);
+}
+
+void Packet::addChecksum(uint32_t checksum) {
+    if (checksumPos + static_cast<int>(sizeof(uint32_t)) > dataStart) {
+        THROW(HdfsIOException,
+              "Packet: failed to add checksum into packet, checksum is too large");
+    }
+
+    WriteBigEndian32ToArray(checksum, &buffer[checksumPos]);
+    checksumPos += checksumSize;
+}
+
+void Packet::addData(const char * buf, int size) {
+    if (size + dataPos > static_cast<int>(buffer.size())) {
+        THROW(HdfsIOException,
+              "Packet: failed add data to packet, packet size is too small");
+    }
+
+    memcpy(&buffer[dataPos], buf, size);
+    dataPos += size;
+    assert(dataPos >= 0);
+}
+
+void Packet::setSyncFlag(bool sync) {
+    syncBlock = sync;
+}
+
+void Packet::increaseNumChunks() {
+    ++numChunks;
+}
+
+bool Packet::isFull() {
+    return numChunks >= maxChunks;
+}
+
+bool Packet::isHeartbeat() {
+    return HEART_BEAT_SEQNO == seqno;
+}
+
+void Packet::setLastPacketInBlock(bool lastPacket) {
+    lastPacketInBlock = lastPacket;
+}
+
+int Packet::getDataSize() {
+    return dataPos - dataStart;
+}
+
+int64_t Packet::getLastByteOffsetBlock() {
+    assert(offsetInBlock >= 0 && dataPos >= dataStart);
+    assert(dataPos - dataStart <= maxChunks * static_cast<int>(buffer.size()));
+    return offsetInBlock + dataPos - dataStart;
+}
+
+const ConstPacketBuffer Packet::getBuffer() {
+    /*
+     * Once this is called, no more data can be added to the packet.
+     * This is called only when the packet is ready to be sent.
+     */
+    int dataLen = dataPos - dataStart;
+    int checksumLen = checksumPos - checksumStart;
+
+    if (checksumPos != dataStart) {
+        /*
+         * move the checksum to cover the gap.
+         * This can happen for the last packet.
+         */
+        memmove(&buffer[dataStart - checksumLen], &buffer[checksumStart],
+                checksumLen);
+        headerStart = dataStart - checksumPos;
+        checksumStart += dataStart - checksumPos;
+        checksumPos = dataStart;
+    }
+
+    assert(dataPos >= 0);
+    int pktLen = dataLen + checksumLen;
+    PacketHeader header(pktLen + sizeof(int32_t)
+                        /* why we add 4 bytes? Because the server will reduce 4 bytes. -_-*/
+                        , offsetInBlock, seqno, lastPacketInBlock, dataLen);
+    header.writeInBuffer(&buffer[headerStart],
+                         PacketHeader::GetPkgHeaderSize());
+    return ConstPacketBuffer(&buffer[headerStart],
+                             PacketHeader::GetPkgHeaderSize() + pktLen);
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h
new file mode 100644
index 0000000..7598344
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h
@@ -0,0 +1,122 @@
+/**
+ * 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_PACKET_H_
+#define _HDFS_LIBHDFS3_CLIENT_PACKET_H_
+
+#include <stdint.h>
+#include <vector>
+
+#define HEART_BEAT_SEQNO -1
+
+namespace hdfs {
+namespace internal {
+
+class ConstPacketBuffer {
+public:
+    ConstPacketBuffer(const char * buf, int size) :
+        buffer(buf), size(size) {
+    }
+
+    const char * getBuffer() const {
+        return buffer;
+    }
+
+    const int getSize() const {
+        return size;
+    }
+
+private:
+    const char * buffer;
+    const int size;
+};
+
+/**
+ * buffer is pointed into like follows:
+ *  (C is checksum data, D is payload data)
+ *
+ * [HHHHHCCCCC________________DDDDDDDDDDDDDDDD___]
+ *       ^    ^               ^               ^
+ *       |    checksumPos     dataStart       dataPos
+ *   checksumStart
+ */
+class Packet {
+public:
+    /**
+     * create a heart beat packet
+     */
+    Packet();
+
+    /**
+     * create a new packet
+     */
+    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 addChecksum(uint32_t checksum);
+
+    void addData(const char * buf, int size);
+
+    void setSyncFlag(bool sync);
+
+    void increaseNumChunks();
+
+    bool isFull();
+
+    bool isHeartbeat();
+
+    void setLastPacketInBlock(bool lastPacket);
+
+    int getDataSize();
+
+    const ConstPacketBuffer getBuffer();
+
+    int64_t getLastByteOffsetBlock();
+
+    int64_t getSeqno() const {
+        return seqno;
+    }
+
+    bool isLastPacketInBlock() const {
+        return lastPacketInBlock;
+    }
+
+    int64_t getOffsetInBlock() const {
+        return offsetInBlock;
+    }
+
+private:
+    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
+    std::vector<char> buffer;
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS3_CLIENT_PACKET_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.cc
new file mode 100644
index 0000000..8c656a3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.cc
@@ -0,0 +1,117 @@
+/**
+ * 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 "BigEndian.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "PacketHeader.h"
+
+namespace hdfs {
+namespace internal {
+
+int PacketHeader::PkgHeaderSize = PacketHeader::CalcPkgHeaderSize();
+
+int PacketHeader::CalcPkgHeaderSize() {
+    PacketHeaderProto header;
+    header.set_offsetinblock(0);
+    header.set_datalen(0);
+    header.set_lastpacketinblock(false);
+    header.set_seqno(0);
+    return header.ByteSize() + sizeof(int32_t) /*packet length*/ + sizeof(int16_t)/* proto length */;
+}
+
+int PacketHeader::GetPkgHeaderSize() {
+    return PkgHeaderSize;
+}
+
+PacketHeader::PacketHeader() :
+    packetLen(0) {
+}
+
+PacketHeader::PacketHeader(int packetLen, int64_t offsetInBlock, int64_t seqno,
+                           bool lastPacketInBlock, int dataLen) :
+    packetLen(packetLen) {
+    proto.set_offsetinblock(offsetInBlock);
+    proto.set_seqno(seqno);
+    proto.set_lastpacketinblock(lastPacketInBlock);
+    proto.set_datalen(dataLen);
+}
+
+int PacketHeader::getDataLen() {
+    return proto.datalen();
+}
+
+bool PacketHeader::isLastPacketInBlock() {
+    return proto.lastpacketinblock();
+}
+
+bool PacketHeader::sanityCheck(int64_t lastSeqNo) {
+    // We should only have a non-positive data length for the last packet
+    if (proto.datalen() <= 0 && !proto.lastpacketinblock())
+        return false;
+
+    // The last packet should not contain data
+    if (proto.lastpacketinblock() && proto.datalen() != 0)
+        return false;
+
+    // Seqnos should always increase by 1 with each packet received
+    if (proto.seqno() != lastSeqNo + 1)
+        return false;
+
+    return true;
+}
+
+int64_t PacketHeader::getSeqno() {
+    return proto.seqno();
+}
+
+int64_t PacketHeader::getOffsetInBlock() {
+    return proto.offsetinblock();
+}
+
+int PacketHeader::getPacketLen() {
+    return packetLen;
+}
+
+void PacketHeader::readFields(const char * buf, size_t size) {
+    int16_t protoLen;
+    assert(size > sizeof(packetLen) + sizeof(protoLen));
+    packetLen = ReadBigEndian32FromArray(buf);
+    protoLen = ReadBigEndian16FromArray(buf + sizeof(packetLen));
+
+    if (packetLen < static_cast<int>(sizeof(int32_t)) || protoLen < 0
+            || static_cast<int>(sizeof(packetLen) + sizeof(protoLen)) + protoLen > static_cast<int>(size)) {
+        THROW(HdfsIOException, "Invalid PacketHeader, packetLen is %d, protoLen is %hd, buf size is %zu", packetLen,
+              protoLen, size);
+    }
+
+    if (!proto.ParseFromArray(buf + sizeof(packetLen) + sizeof(protoLen),
+                              protoLen)) {
+        THROW(HdfsIOException,
+              "PacketHeader cannot parse PacketHeaderProto from datanode response.");
+    }
+}
+
+void PacketHeader::writeInBuffer(char * buf, size_t size) {
+    buf = WriteBigEndian32ToArray(packetLen, buf);
+    buf = WriteBigEndian16ToArray(proto.ByteSize(), buf);
+    proto.SerializeToArray(buf, size - sizeof(int32_t) - sizeof(int16_t));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h
new file mode 100644
index 0000000..f8447b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h
@@ -0,0 +1,61 @@
+/**
+ * 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_PACKETHEADER_H_
+#define _HDFS_LIBHDFS3_CLIENT_PACKETHEADER_H_
+
+#include "datatransfer.pb.h"
+
+using hadoop::hdfs::PacketHeaderProto;
+
+namespace hdfs {
+namespace internal {
+
+class PacketHeader {
+public:
+    PacketHeader();
+    PacketHeader(int packetLen, int64_t offsetInBlock, int64_t seqno,
+                 bool lastPacketInBlock, int dataLen);
+    bool isLastPacketInBlock();
+    bool sanityCheck(int64_t lastSeqNo);
+    int getDataLen();
+    int getPacketLen();
+    int64_t getOffsetInBlock();
+    int64_t getSeqno();
+    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();
+
+private:
+    static int PkgHeaderSize;
+private:
+    int32_t packetLen;
+    PacketHeaderProto proto;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_PACKETHEADER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.cc
new file mode 100644
index 0000000..09d76c3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.cc
@@ -0,0 +1,38 @@
+/**
+ * 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 "Permission.h"
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+namespace hdfs {
+
+Permission::Permission(uint16_t mode) {
+    if (mode >> 10) {
+        THROW(InvalidParameter,
+              "Invalid parameter: cannot convert %u to \"Permission\"",
+              static_cast<unsigned int>(mode));
+    }
+
+    userAction = (Action)((mode >> 6) & 7);
+    groupAction = (Action)((mode >> 3) & 7);
+    otherAction = (Action)(mode & 7);
+    stickyBit = (((mode >> 9) & 1) == 1);
+}
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.h
new file mode 100644
index 0000000..fafb7fe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.h
@@ -0,0 +1,215 @@
+/**
+ * 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_PERMISSION_H_
+#define _HDFS_LIBHDFS3_CLIENT_PERMISSION_H_
+
+#include <string>
+
+namespace hdfs {
+
+/**
+ * Action is used to describe a action the user is permitted to apply on a file.
+ */
+enum Action {
+    NONE, //("---"),
+    EXECUTE, //("--x"),
+    WRITE, //("-w-"),
+    WRITE_EXECUTE, //("-wx"),
+    READ, //("r--"),
+    READ_EXECUTE, //("r-x"),
+    READ_WRITE, //("rw-"),
+    ALL //("rwx");
+};
+
+/**
+ * To test Action a if implies Action b
+ * @param a Action to be tested.
+ * @param b Action target.
+ * @return return true if a implies b.
+ */
+static inline bool implies(const Action &a, const Action &b) {
+    return (a & b) == b;
+}
+
+/**
+ * To construct a new Action using a and b
+ * @param a Action to be used.
+ * @param b Action to be used.
+ * @return return a new Action.
+ */
+static inline Action operator &(const Action &a, const Action &b) {
+    return (Action)(((unsigned int) a) & (unsigned int) b);
+}
+/**
+ * To construct a new Action using a or b
+ * @param a Action to be used.
+ * @param b Action to be used.
+ * @return return a new Action.
+ */
+static inline Action operator |(const Action &a, const Action &b) {
+    return (Action)(((unsigned int) a) | (unsigned int) b);
+}
+/**
+ * To construct a new Action of complementary of a given Action
+ * @param a Action to be used.
+ * @return return a new Action
+ */
+static inline Action operator ~(const Action &a) {
+    return (Action)(7 - (unsigned int) a);
+}
+
+/**
+ * To convert a Action to a readable string.
+ * @param a the Action to be convert.
+ * @return a readable string
+ */
+static inline std::string toString(const Action &a) {
+    switch (a) {
+    case NONE:
+        return "---";
+
+    case EXECUTE:
+        return "--x";
+
+    case WRITE:
+        return "-w-";
+
+    case WRITE_EXECUTE:
+        return "-wx";
+
+    case READ:
+        return "r--";
+
+    case READ_EXECUTE:
+        return "r-x";
+
+    case READ_WRITE:
+        return "rw-";
+
+    case ALL:
+        return "rwx";
+    }
+}
+
+/**
+ * Permission is used to describe a file permission.
+ */
+class Permission {
+public:
+    /**
+     * To construct a Permission.
+     * @param u owner permission.
+     * @param g group permission.
+     * @param o other user permission.
+     */
+    Permission(const Action &u, const Action &g, const Action &o) :
+        userAction(u), groupAction(g), otherAction(o), stickyBit(false) {
+    }
+
+    /**
+     * To construct a Permission from a uint16.
+     * @param mode permission flag.
+     */
+    Permission(uint16_t mode);
+
+public:
+    /**
+     * To get group permission
+     * @return the group permission
+     */
+    Action getGroupAction() const {
+        return groupAction;
+    }
+
+    /**
+     * To set group permission
+     * @param groupAction the group permission
+     */
+    void setGroupAction(Action groupAction) {
+        this->groupAction = groupAction;
+    }
+
+    /**
+     * To get other user permission
+     * @return other user permission
+     */
+    Action getOtherAction() const {
+        return otherAction;
+    }
+
+    /**
+     * To set other user permission
+     * @param otherAction other user permission
+     */
+    void setOtherAction(Action otherAction) {
+        this->otherAction = otherAction;
+    }
+
+    /**
+     * To get owner permission
+     * @return the owner permission
+     */
+    Action getUserAction() const {
+        return userAction;
+    }
+
+    /**
+     * To set owner permission
+     * @param userAction the owner permission
+     */
+    void setUserAction(Action userAction) {
+        this->userAction = userAction;
+    }
+
+    /**
+     * To convert a Permission to a readable string
+     * @return a readable string
+     */
+    std::string toString() const {
+        return hdfs::toString(userAction) + hdfs::toString(groupAction)
+               + hdfs::toString(otherAction);
+    }
+
+    /**
+     * To convert a Permission to a uint16 flag
+     * @return a uint16 flag
+     */
+    uint16_t toShort() const {
+        return (uint16_t)((((uint16_t) userAction) << 6)
+                          + (((uint16_t) groupAction) << 3) + (((uint16_t) otherAction))
+                          + ((stickyBit ? 1 << 9 : 0)));
+    }
+
+    bool operator ==(const Permission &other) const {
+        return userAction == other.userAction
+               && groupAction == other.groupAction
+               && otherAction == other.otherAction
+               && stickyBit == other.stickyBit;
+    }
+
+private:
+    Action userAction;
+    Action groupAction;
+    Action otherAction;
+
+    bool stickyBit;
+};
+
+}
+
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.cc
new file mode 100644
index 0000000..02a2ddd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.cc
@@ -0,0 +1,375 @@
+/**
+ * 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 "BigEndian.h"
+#include "DataTransferProtocolSender.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "HWCrc32c.h"
+#include "RemoteBlockReader.h"
+#include "SWCrc32c.h"
+#include "WriteBuffer.h"
+#include "datatransfer.pb.h"
+
+#include <inttypes.h>
+#include <vector>
+
+using hadoop::hdfs::ClientReadStatusProto;
+using hadoop::hdfs::BlockOpResponseProto;
+using hadoop::hdfs::ChecksumProto;
+using hadoop::hdfs::ChecksumTypeProto;
+using hadoop::hdfs::ReadOpChecksumInfoProto;
+using hadoop::hdfs::Status;
+
+namespace hdfs {
+namespace internal {
+
+RemoteBlockReader::RemoteBlockReader(const ExtendedBlock &eb,
+          DatanodeInfo &datanode, int64_t start, int64_t len,
+          const Token &token, const char * clientName, bool verify,
+          SessionConfig &conf) :
+    verify(verify), datanode(datanode), binfo(eb), checksumSize(0),
+      chunkSize(0), position(0), size(0), cursor(
+        start), endOffset(len + start), lastSeqNo(-1) {
+    try {
+        assert(start >= 0);
+        readTimeout = conf.getInputReadTimeout();
+        writeTimeout = conf.getInputWriteTimeout();
+        connTimeout = conf.getInputConnTimeout();
+        sock = shared_ptr<Socket>(new TcpSocketImpl());
+        in = shared_ptr<BufferedSocketReader>(
+                      new BufferedSocketReaderImpl(*sock));
+        sock->connect(datanode.getIpAddr().c_str(), datanode.getXferPort(),
+                      connTimeout);
+        sender = shared_ptr<DataTransferProtocol>(
+                     new DataTransferProtocolSender(*sock, writeTimeout,
+                                                    datanode.formatAddress()));
+        sender->readBlock(eb, token, clientName, start, len);
+        checkResponse();
+    } catch (const HdfsTimeoutException &e) {
+        NESTED_THROW(HdfsIOException,
+                     "RemoteBlockReader: Failed to setup remote block reader "
+                     "for block %s from node %s",
+                     eb.toString().c_str(), datanode.formatAddress().c_str());
+    }
+}
+
+RemoteBlockReader::~RemoteBlockReader() {
+    sock->close();
+}
+
+void RemoteBlockReader::checkResponse() {
+    std::vector<char> respBuffer;
+    int32_t respSize = in->readVarint32(readTimeout);
+
+    if (respSize <= 0 || respSize > 10 * 1024 * 1024) {
+        THROW(HdfsIOException, "RemoteBlockReader get a invalid response "
+              "size: %d, Block: %s, from Datanode: %s",
+              respSize, binfo.toString().c_str(),
+              datanode.formatAddress().c_str());
+    }
+
+    respBuffer.resize(respSize);
+    in->readFully(&respBuffer[0], respSize, readTimeout);
+    BlockOpResponseProto resp;
+
+    if (!resp.ParseFromArray(&respBuffer[0], respBuffer.size())) {
+        THROW(HdfsIOException, "RemoteBlockReader cannot parse "
+              "BlockOpResponseProto from Datanode response, "
+              "Block: %s, from Datanode: %s", binfo.toString().c_str(),
+              datanode.formatAddress().c_str());
+    }
+
+    if (resp.status() != hadoop::hdfs::SUCCESS) {
+        std::string msg;
+
+        if (resp.has_message()) {
+            msg = resp.message();
+        }
+
+        if (resp.status() == hadoop::hdfs::ERROR_ACCESS_TOKEN) {
+            THROW(HdfsInvalidBlockToken, "RemoteBlockReader: block's token "
+                  "is invalid. Datanode: %s, Block: %s",
+                  datanode.formatAddress().c_str(), binfo.toString().c_str());
+        } else {
+            THROW(HdfsIOException,
+                  "RemoteBlockReader: Datanode return an error when sending "
+                  "read request to Datanode: %s, Block: %s, %s.",
+                  datanode.formatAddress().c_str(), binfo.toString().c_str(),
+                  (msg.empty() ? "check Datanode's log for more information" :
+                     msg.c_str()));
+        }
+    }
+
+    const ReadOpChecksumInfoProto &checksumInfo = resp.readopchecksuminfo();
+    const ChecksumProto &cs = checksumInfo.checksum();
+    chunkSize = cs.bytesperchecksum();
+
+    if (chunkSize < 0) {
+        THROW(HdfsIOException,
+              "RemoteBlockReader invalid chunk size: %d, expected range[0, %"
+              PRId64 "], Block: %s, from Datanode: %s",
+              chunkSize, binfo.getNumBytes(), binfo.toString().c_str(),
+              datanode.formatAddress().c_str());
+    }
+
+    switch (cs.type()) {
+    case ChecksumTypeProto::CHECKSUM_NULL:
+        verify = false;
+        checksumSize = 0;
+        break;
+
+    case ChecksumTypeProto::CHECKSUM_CRC32:
+        THROW(HdfsIOException, "RemoteBlockReader does not support CRC32 "
+              "checksum, Block: %s, from Datanode: %s",
+              binfo.toString().c_str(), datanode.formatAddress().c_str());
+        break;
+
+    case ChecksumTypeProto::CHECKSUM_CRC32C:
+        if (HWCrc32c::available()) {
+            checksum = shared_ptr<Checksum>(new HWCrc32c());
+        } else {
+            checksum = shared_ptr<Checksum>(new SWCrc32c());
+        }
+
+        checksumSize = sizeof(int32_t);
+        break;
+
+    default:
+        THROW(HdfsIOException, "RemoteBlockReader cannot recognize checksum "
+              "type: %d, Block: %s, from Datanode: %s",
+              static_cast<int>(cs.type()), binfo.toString().c_str(),
+              datanode.formatAddress().c_str());
+    }
+
+    /*
+     * The offset into the block at which the first packet
+     * will start. This is necessary since reads will align
+     * backwards to a checksum chunk boundary.
+     */
+    int64_t firstChunkOffset = checksumInfo.chunkoffset();
+
+    if (firstChunkOffset < 0 || firstChunkOffset > cursor ||
+          firstChunkOffset <= cursor - chunkSize) {
+        THROW(HdfsIOException,
+              "RemoteBlockReader invalid first chunk offset: %" PRId64
+              ", expected range[0, %" PRId64 "], " "Block: %s, from Datanode: %s",
+              firstChunkOffset, cursor, binfo.toString().c_str(),
+              datanode.formatAddress().c_str());
+    }
+}
+
+shared_ptr<PacketHeader> RemoteBlockReader::readPacketHeader() {
+    try {
+        shared_ptr<PacketHeader> retval;
+        static const int packetHeaderLen = PacketHeader::GetPkgHeaderSize();
+        std::vector<char> buf(packetHeaderLen);
+
+        if (lastHeader && lastHeader->isLastPacketInBlock()) {
+            THROW(HdfsIOException, "RemoteBlockReader: read over block end "
+                 "from Datanode: %s, Block: %s.",
+                 datanode.formatAddress().c_str(), binfo.toString().c_str());
+        }
+
+        in->readFully(&buf[0], packetHeaderLen, readTimeout);
+        retval = shared_ptr<PacketHeader>(new PacketHeader);
+        retval->readFields(&buf[0], packetHeaderLen);
+        return retval;
+    } catch (const HdfsIOException &e) {
+        NESTED_THROW(HdfsIOException, "RemoteBlockReader: failed to read "
+                 "block header for Block: %s from Datanode: %s.",
+                 binfo.toString().c_str(), datanode.formatAddress().c_str());
+    }
+}
+
+void RemoteBlockReader::readNextPacket() {
+    assert(position >= size);
+    lastHeader = readPacketHeader();
+    int dataSize = lastHeader->getDataLen();
+    int64_t pendingAhead = 0;
+
+    if (!lastHeader->sanityCheck(lastSeqNo)) {
+        THROW(HdfsIOException, "RemoteBlockReader: Packet failed on sanity "
+              "check for block %s from Datanode %s.",
+              binfo.toString().c_str(), datanode.formatAddress().c_str());
+    }
+
+    assert(dataSize > 0 || lastHeader->getPacketLen() == sizeof(int32_t));
+
+    if (dataSize > 0) {
+        int chunks = (dataSize + chunkSize - 1) / chunkSize;
+        int checksumLen = chunks * checksumSize;
+        size = checksumLen + dataSize;
+        assert(size ==
+             lastHeader->getPacketLen() - static_cast<int>(sizeof(int32_t)));
+        buffer.resize(size);
+        in->readFully(&buffer[0], size, readTimeout);
+        lastSeqNo = lastHeader->getSeqno();
+
+        if (lastHeader->getPacketLen() != static_cast<int>(sizeof(int32_t)) +
+            dataSize + checksumLen) {
+            THROW(HdfsIOException, "Invalid Packet, packetLen is %d, "
+                  "dataSize is %d, checksum size is %d",
+                  lastHeader->getPacketLen(), dataSize, checksumLen);
+        }
+
+        if (verify) {
+            verifyChecksum(chunks);
+        }
+
+        /*
+         * skip checksum
+         */
+        position = checksumLen;
+        /*
+         * the first packet we get may start at the position before we required
+         */
+        pendingAhead = cursor - lastHeader->getOffsetInBlock();
+        pendingAhead = pendingAhead > 0 ? pendingAhead : 0;
+        position += pendingAhead;
+    }
+
+    /*
+     * we reach the end of the range we required, send status to datanode
+     * if datanode do not sending data anymore.
+     */
+
+    if (cursor + dataSize - pendingAhead >= endOffset && readTrailingEmptyPacket()) {
+        sendStatus();
+    }
+}
+
+bool RemoteBlockReader::readTrailingEmptyPacket() {
+    shared_ptr<PacketHeader> trailingHeader = readPacketHeader();
+
+    if (!trailingHeader->isLastPacketInBlock() || trailingHeader->getDataLen() != 0) {
+        return false;
+    }
+
+    return true;
+}
+
+void RemoteBlockReader::sendStatus() {
+    ClientReadStatusProto status;
+
+    if (verify) {
+        status.set_status(hadoop::hdfs::CHECKSUM_OK);
+    } else {
+        status.set_status(hadoop::hdfs::SUCCESS);
+    }
+
+    WriteBuffer buffer;
+    int size = status.ByteSize();
+    buffer.writeVarint32(size);
+    status.SerializeToArray(buffer.alloc(size), size);
+    sock->writeFully(buffer.getBuffer(0), buffer.getDataSize(0), writeTimeout);
+}
+
+void RemoteBlockReader::verifyChecksum(int chunks) {
+    int dataSize = lastHeader->getDataLen();
+    char * pchecksum = &buffer[0];
+    char * pdata = &buffer[0] + (chunks * checksumSize);
+
+    for (int i = 0; i < chunks; ++i) {
+        int size = chunkSize < dataSize ? chunkSize : dataSize;
+        dataSize -= size;
+        checksum->reset();
+        checksum->update(pdata + (i * chunkSize), size);
+        uint32_t result = checksum->getValue();
+        uint32_t target =
+          ReadBigEndian32FromArray(pchecksum + (i * checksumSize));
+
+        if (result != target && size == chunkSize) {
+            THROW(ChecksumException, "RemoteBlockReader: checksum not match "
+                  "for Block: %s, on Datanode: %s",
+                  binfo.toString().c_str(), datanode.formatAddress().c_str());
+        }
+    }
+
+    assert(0 == dataSize);
+}
+
+int64_t RemoteBlockReader::available() {
+    return size - position > 0 ? size - position : 0;
+}
+
+int32_t RemoteBlockReader::read(char * buf, int32_t len) {
+    assert(0 != len && NULL != buf);
+
+    if (cursor >= endOffset) {
+        THROW(HdfsIOException, "RemoteBlockReader: read over block end from "
+              "Datanode: %s, Block: %s.",
+              datanode.formatAddress().c_str(), binfo.toString().c_str());
+    }
+
+    try {
+        if (position >= size) {
+            readNextPacket();
+        }
+
+        int32_t todo = len < size - position ? len : size - position;
+        memcpy(buf, &buffer[position], todo);
+        position += todo;
+        cursor += todo;
+        return todo;
+    } catch (const HdfsTimeoutException &e) {
+        NESTED_THROW(HdfsIOException, "RemoteBlockReader: failed to read "
+                  "Block: %s from Datanode: %s.",
+                  binfo.toString().c_str(), datanode.formatAddress().c_str());
+    } catch (const HdfsNetworkException &e) {
+        NESTED_THROW(HdfsIOException, "RemoteBlockReader: failed to read "
+                  "Block: %s from Datanode: %s.",
+                  binfo.toString().c_str(), datanode.formatAddress().c_str());
+    }
+}
+
+void RemoteBlockReader::skip(int64_t len) {
+    int64_t todo = len;
+    assert(cursor + len <= endOffset);
+
+    try {
+        while (todo > 0) {
+            if (cursor >= endOffset) {
+                THROW(HdfsIOException, "RemoteBlockReader: skip over block "
+                  "end from Datanode: %s, Block: %s.",
+                  datanode.formatAddress().c_str(), binfo.toString().c_str());
+            }
+
+            if (position >= size) {
+                readNextPacket();
+            }
+
+            int batch = size - position;
+            batch = batch < todo ? batch : static_cast<int>(todo);
+            position += batch;
+            cursor += batch;
+            todo -= batch;
+        }
+    } catch (const HdfsTimeoutException &e) {
+        NESTED_THROW(HdfsIOException, "RemoteBlockReader: failed to read "
+                 "Block: %s from Datanode: %s.",
+                 binfo.toString().c_str(), datanode.formatAddress().c_str());
+    } catch (const HdfsNetworkException &e) {
+        NESTED_THROW(HdfsIOException, "RemoteBlockReader: failed to read "
+                 "Block: %s from Datanode: %s.",
+                 binfo.toString().c_str(), datanode.formatAddress().c_str());
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h
new file mode 100644
index 0000000..548118b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h
@@ -0,0 +1,100 @@
+/**
+ * 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_REMOTEBLOCKREADER_H_
+#define _HDFS_LIBHDFS3_CLIENT_REMOTEBLOCKREADER_H_
+
+#include "BlockReader.h"
+#include "Checksum.h"
+#include "DataTransferProtocol.h"
+#include "PacketHeader.h"
+#include "SessionConfig.h"
+#include "common/SharedPtr.h"
+#include "network/BufferedSocketReader.h"
+#include "network/TcpSocket.h"
+#include "server/DatanodeInfo.h"
+#include "server/LocatedBlocks.h"
+
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+class RemoteBlockReader: public BlockReader {
+public:
+    RemoteBlockReader(const ExtendedBlock &eb, DatanodeInfo &datanode,
+                      int64_t start, int64_t len, const Token &token,
+                      const char *clientName, bool verify, SessionConfig &conf);
+
+    ~RemoteBlockReader();
+
+    /**
+     * Get how many bytes can be read without blocking.
+     * @return The number of bytes can be read without blocking.
+     */
+    virtual int64_t available();
+
+    /**
+     * To read data from block.
+     * @param buf the buffer used to filled.
+     * @param size the number of bytes to be read.
+     * @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 len);
+
+    /**
+     * Move the cursor forward len bytes.
+     * @param len The number of bytes to skip.
+     */
+    virtual void skip(int64_t len);
+
+private:
+    bool readTrailingEmptyPacket();
+    shared_ptr<PacketHeader> readPacketHeader();
+    void checkResponse();
+    void readNextPacket();
+    void sendStatus();
+    void verifyChecksum(int chunks);
+
+private:
+    bool verify; //verify checksum or not.
+    DatanodeInfo &datanode;
+    const ExtendedBlock &binfo;
+    int checksumSize;
+    int chunkSize;
+    int connTimeout;
+    int position; //point in buffer.
+    int readTimeout;
+    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
+    shared_ptr<BufferedSocketReader> in;
+    shared_ptr<Checksum> checksum;
+    shared_ptr<DataTransferProtocol> sender;
+    shared_ptr<PacketHeader> lastHeader;
+    shared_ptr<Socket> sock;
+    std::vector<char> buffer;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_REMOTEBLOCKREADER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.cc
new file mode 100644
index 0000000..16ef93f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.cc
@@ -0,0 +1,172 @@
+/**
+ * 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 "Token.h"
+#include "WritableUtils.h"
+
+#include <gsasl.h>
+#include <string>
+#include <vector>
+
+using namespace hdfs::internal;
+
+namespace hdfs {
+namespace internal {
+
+static std::string Base64Encode(const char *input, size_t len) {
+    int rc = 0;
+    size_t outLen;
+    char * output = NULL;
+    std::string retval;
+
+    if (GSASL_OK != (rc = gsasl_base64_to(input, len, &output, &outLen))) {
+        assert(GSASL_MALLOC_ERROR == rc);
+        throw std::bad_alloc();
+    }
+
+    assert(NULL != output);
+    retval = output;
+    gsasl_free(output);
+
+    for (size_t i = 0 ; i < retval.length(); ++i) {
+        switch (retval[i]) {
+        case '+':
+            retval[i] = '-';
+            break;
+
+        case '/':
+            retval[i] = '_';
+            break;
+
+        case '=':
+            retval.resize(i);
+            break;
+
+        default:
+            break;
+        }
+    }
+
+    return retval;
+}
+
+static void Base64Decode(const std::string &urlSafe,
+                         std::vector<char> &buffer) {
+    int retval = 0, append = 0;
+    size_t outLen;
+    char * output = NULL;
+    std::string input = urlSafe;
+
+    for (size_t i = 0; i < input.length(); ++i) {
+        switch (input[i]) {
+        case '-':
+            input[i] = '+';
+            break;
+
+        case '_':
+            input[i] = '/';
+            break;
+
+        default:
+            break;
+        }
+    }
+
+    while (true) {
+        retval = gsasl_base64_from(&input[0], input.length(), &output, &outLen);
+
+        if (GSASL_OK != retval) {
+            switch (retval) {
+            case GSASL_BASE64_ERROR:
+                if (append++ < 2) {
+                    input.append("=");
+                    continue;
+                }
+
+                throw std::invalid_argument(
+                    "invalid input of gsasl_base64_from");
+
+            case GSASL_MALLOC_ERROR:
+                throw std::bad_alloc();
+
+            default:
+                assert(
+                    false
+                    && "unexpected return value from gsasl_base64_from");
+            }
+        }
+
+        break;
+    }
+
+    assert(outLen >= 0);
+    buffer.resize(outLen);
+    memcpy(&buffer[0], output, outLen);
+    gsasl_free(output);
+}
+
+std::string Token::toString() const {
+    try {
+        size_t len = 0;
+        std::vector<char> buffer(1024);
+        WritableUtils out(&buffer[0], buffer.size());
+        len += out.WriteInt32(identifier.size());
+        len += out.WriteRaw(&identifier[0], identifier.size());
+        len += out.WriteInt32(password.size());
+        len += out.WriteRaw(&password[0], password.size());
+        len += out.WriteText(kind);
+        len += out.WriteText(service);
+        return Base64Encode(&buffer[0], len);
+    } catch (...) {
+        NESTED_THROW(HdfsIOException, "cannot convert token to string");
+    }
+}
+
+void Token::fromString(const std::string &str) {
+    int32_t len;
+
+    try {
+        std::vector<char> buffer;
+        Base64Decode(str, buffer);
+        WritableUtils in(&buffer[0], buffer.size());
+        len = in.ReadInt32();
+        identifier.resize(len);
+        in.ReadRaw(&identifier[0], len);
+        len = in.ReadInt32();
+        password.resize(len);
+        in.ReadRaw(&password[0], len);
+        kind = in.ReadText();
+        service = in.ReadText();
+    } catch (...) {
+        NESTED_THROW(HdfsInvalidBlockToken,
+                     "cannot construct a token from the string");
+    }
+}
+
+size_t Token::hash_value() const {
+    size_t values[] = { StringHasher(identifier), StringHasher(password),
+                        StringHasher(kind), StringHasher(service)
+                      };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.h
new file mode 100644
index 0000000..c72cd86
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.h
@@ -0,0 +1,82 @@
+/**
+ * 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_TOKEN_H_
+#define _HDFS_LIBHDFS3_CLIENT_TOKEN_H_
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class Token {
+public:
+    std::string getIdentifier() const {
+        return identifier;
+    }
+
+    void setIdentifier(const std::string &identifier) {
+        this->identifier = identifier;
+    }
+
+    std::string getKind() const {
+        return kind;
+    }
+
+    void setKind(const std::string &kind) {
+        this->kind = kind;
+    }
+
+    std::string getPassword() const {
+        return password;
+    }
+
+    void setPassword(const std::string &password) {
+        this->password = password;
+    }
+
+    std::string getService() const {
+        return service;
+    }
+
+    void setService(const std::string &service) {
+        this->service = service;
+    }
+
+    bool operator ==(const Token &other) const {
+        return identifier == other.identifier && password == other.password
+               && kind == other.kind && service == other.service;
+    }
+
+    std::string toString() const;
+
+    void fromString(const std::string &str);
+
+    size_t hash_value() const;
+
+private:
+    std::string identifier;
+    std::string password;
+    std::string kind;
+    std::string service;
+};
+
+}
+}
+
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.cc
new file mode 100644
index 0000000..a68bca0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.cc
@@ -0,0 +1,72 @@
+/**
+ * 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 "UserInfo.h"
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+#include <pwd.h>
+#include <sys/types.h>
+#include <unistd.h>
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+UserInfo UserInfo::LocalUser() {
+    UserInfo retval;
+    uid_t uid, euid;
+    int bufsize;
+    struct passwd pwd, epwd, *result = NULL;
+    euid = geteuid();
+    uid = getuid();
+
+    if ((bufsize = sysconf(_SC_GETPW_R_SIZE_MAX)) == -1) {
+        THROW(InvalidParameter,
+              "Invalid input: \"sysconf\" function failed to get the "
+              "configure with key \"_SC_GETPW_R_SIZE_MAX\".");
+    }
+
+    std::vector<char> buffer(bufsize);
+
+    if (getpwuid_r(euid, &epwd, &buffer[0], bufsize, &result) != 0 || !result) {
+        THROW(InvalidParameter,
+              "Invalid input: effective user name cannot be found with UID %u.",
+              euid);
+    }
+
+    retval.setEffectiveUser(epwd.pw_name);
+
+    if (getpwuid_r(uid, &pwd, &buffer[0], bufsize, &result) != 0 || !result) {
+        THROW(InvalidParameter,
+              "Invalid input: real user name cannot be found with UID %u.",
+              uid);
+    }
+
+    retval.setRealUser(pwd.pw_name);
+    return retval;
+}
+
+size_t UserInfo::hash_value() const {
+    size_t values[] = { StringHasher(realUser), effectiveUser.hash_value() };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.h
new file mode 100644
index 0000000..efc2c60
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.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_USERINFO_H_
+#define _HDFS_LIBHDFS3_CLIENT_USERINFO_H_
+
+#include "Hash.h"
+#include "KerberosName.h"
+#include "Logger.h"
+#include "Token.h"
+
+#include <map>
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class UserInfo {
+public:
+    UserInfo() {
+    }
+
+    explicit UserInfo(const std::string &u) :
+        effectiveUser(u) {
+    }
+
+    const std::string &getRealUser() const {
+        return realUser;
+    }
+
+    void setRealUser(const std::string &user) {
+        this->realUser = user;
+    }
+
+    const std::string &getEffectiveUser() const {
+        return effectiveUser.getName();
+    }
+
+    void setEffectiveUser(const std::string &effectiveUser) {
+        this->effectiveUser = KerberosName(effectiveUser);
+    }
+
+    std::string getPrincipal() const {
+        return effectiveUser.getPrincipal();
+    }
+
+    bool operator ==(const UserInfo &other) const {
+        return realUser == other.realUser
+               && effectiveUser == other.effectiveUser;
+    }
+
+    void addToken(const Token &token) {
+        tokens[std::make_pair(token.getKind(), token.getService())] = token;
+    }
+
+    const Token * selectToken(const std::string &kind,
+                              const std::string &service) const {
+        std::map<std::pair<std::string, std::string>,
+            Token>::const_iterator it;
+        it = tokens.find(std::make_pair(kind, service));
+        if (it == tokens.end()) {
+            return NULL;
+        }
+        return &it->second;
+    }
+
+    size_t hash_value() const;
+
+public:
+    static UserInfo LocalUser();
+
+private:
+    KerberosName effectiveUser;
+    std::map<std::pair<std::string, std::string>, Token> tokens;
+    std::string realUser;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::UserInfo);
+
+#endif /* _HDFS_LIBHDFS3_CLIENT_USERINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
index a434aaf..fd62d5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
@@ -20,7 +20,7 @@
 #define _HDFS_LIBHDFS3_COMMON_LRUMAP_H_
 
 #include "Thread.h"
-#include "Unordered.h"
+#include "UnorderedMap.h"
 
 #include <list>
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SharedPtr.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SharedPtr.h
index 8e0a40e..76ab1d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SharedPtr.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SharedPtr.h
@@ -19,6 +19,18 @@
 #ifndef _HDFS_LIBHDFS3_COMMON_SHARED_PTR_H_
 #define _HDFS_LIBHDFS3_COMMON_SHARED_PTR_H_
 
+#ifdef _LIBCPP_VERSION
+#include <memory>
+
+namespace hdfs {
+namespace internal {
+
+using std::shared_ptr;
+
+}
+}
+
+#else
 #include <tr1/memory>
 
 namespace hdfs {
@@ -30,3 +42,4 @@
 }
 
 #endif
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StackPrinter.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StackPrinter.h
index 4dff889..00e3a2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StackPrinter.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StackPrinter.h
@@ -23,7 +23,7 @@
 
 #include <string>
 
-#ifndef DEFAULT_STACK_PREFIX 
+#ifndef DEFAULT_STACK_PREFIX
 #define DEFAULT_STACK_PREFIX "\t@\t"
 #endif
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h
index 3bb08af..8c2c549 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h
@@ -19,7 +19,21 @@
 #ifndef _HDFS_LIBHDFS3_COMMON_UNORDERED_MAP_H_
 #define _HDFS_LIBHDFS3_COMMON_UNORDERED_MAP_H_
 
-#include <tr1/unordred_map>
+#ifdef _LIBCPP_VERSION
+
+#include <unordered_map>
+
+namespace hdfs {
+namespace internal {
+
+using std::unordered_map;
+
+}
+}
+
+#else
+
+#include <tr1/unordered_map>
 
 namespace hdfs {
 namespace internal {
@@ -30,3 +44,4 @@
 }
 
 #endif
+#endif
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.cc
new file mode 100644
index 0000000..fe30d680
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.cc
@@ -0,0 +1,124 @@
+/**
+ * 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 "BufferedSocketReader.h"
+#include "DateTime.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+#include <google/protobuf/io/coded_stream.h>
+
+using namespace google::protobuf::io;
+
+namespace hdfs {
+namespace internal {
+
+BufferedSocketReaderImpl::BufferedSocketReaderImpl(Socket & s) :
+    cursor(0), size(0), sock(s), buffer(sizeof(int64_t)) {
+}
+
+int32_t BufferedSocketReaderImpl::read(char * b, int32_t s) {
+    assert(s > 0 && NULL != b);
+    int32_t done = s < size - cursor ? s : size - cursor;
+
+    if (done > 0) {
+        memcpy(b, &buffer[cursor], done);
+        cursor += done;
+        return done;
+    } else {
+        assert(size == cursor);
+        size = cursor = 0;
+        return sock.read(b, s);
+    }
+}
+
+void BufferedSocketReaderImpl::readFully(char * b, int32_t s, int timeout) {
+    assert(s > 0 && NULL != b);
+    int32_t done = s < size - cursor ? s : size - cursor;
+    memcpy(b, &buffer[cursor], done);
+    cursor += done;
+
+    if (done < s) {
+        assert(size == cursor);
+        size = cursor = 0;
+        sock.readFully(b + done, s - done, timeout);
+    }
+}
+
+int32_t BufferedSocketReaderImpl::readBigEndianInt32(int timeout) {
+    char buf[sizeof(int32_t)];
+    readFully(buf, sizeof(buf), timeout);
+    return ntohl(*reinterpret_cast<int32_t *>(buf));
+}
+
+int32_t BufferedSocketReaderImpl::readVarint32(int timeout) {
+    int32_t value;
+    bool rc = false;
+    int deadline = timeout;
+    memmove(&buffer[0], &buffer[cursor], size - cursor);
+    size -= cursor;
+    cursor = 0;
+
+    while (!rc) {
+        CodedInputStream in(reinterpret_cast<uint8_t *>(&buffer[cursor]),
+                            size - cursor);
+        in.PushLimit(size - cursor);
+        rc = in.ReadVarint32(reinterpret_cast<uint32_t *>(&value));
+
+        if (rc) {
+            cursor += size - cursor - in.BytesUntilLimit();
+            return value;
+        }
+
+        steady_clock::time_point s = steady_clock::now();
+        CheckOperationCanceled();
+
+        if (size == static_cast<int32_t>(buffer.size())) {
+            THROW(HdfsNetworkException,
+                  "Invalid varint type or buffer is too small, buffer size = %d.",
+                  static_cast<int>(buffer.size()));
+        }
+
+        if (sock.poll(true, false, deadline)) {
+            size += sock.read(&buffer[size], buffer.size() - size);
+        }
+
+        steady_clock::time_point e = steady_clock::now();
+
+        if (timeout > 0) {
+            deadline -= ToMilliSeconds(s, e);
+        }
+
+        if (timeout >= 0 && deadline <= 0) {
+            THROW(HdfsTimeoutException, "Read %d bytes timeout", size);
+        }
+    }
+
+    return 0;
+}
+
+bool BufferedSocketReaderImpl::poll(int timeout) {
+    if (cursor < size) {
+        return true;
+    }
+
+    return sock.poll(true, false, timeout);
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.h
new file mode 100644
index 0000000..efe7826
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.h
@@ -0,0 +1,128 @@
+/**
+ * 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_NETWORK_BUFFEREDSOCKET_H_
+#define _HDFS_LIBHDFS3_NETWORK_BUFFEREDSOCKET_H_
+
+#include <vector>
+#include <stdint.h>
+#include <cstdlib>
+
+#include "Socket.h"
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A warper of Socket, read big endian int and varint from socket.
+ */
+class BufferedSocketReader {
+public:
+    virtual ~BufferedSocketReader() {
+    }
+
+    /**
+     * Read data from socket, if there is data buffered, read from buffer first.
+     * If there is nothing can be read, the caller will be blocked.
+     * @param b The buffer used to receive data.
+     * @param s The size of bytes to be read.
+     * @return The size of data already read.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     */
+    virtual int32_t read(char * b, int32_t s) = 0;
+
+    /**
+     * Read data form socket, if there is data buffered, read from buffer first.
+     * If there is not enough data can be read, the caller will be blocked.
+     * @param b The buffer used to receive data.
+     * @param s The size of bytes to read.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     * @throw HdfsTimeout
+     */
+    virtual void readFully(char * b, int32_t s, int timeout) = 0;
+
+    /**
+     * Read a 32 bit big endian integer from socket.
+     * If there is not enough data can be read, the caller will be blocked.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @return A 32 bit integer.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     * @throw HdfsTimeout
+     */
+    virtual int32_t readBigEndianInt32(int timeout) = 0;
+
+    /**
+     * Read a variable length encoding 32bit integer from socket.
+     * If there is not enough data can be read, the caller will be blocked.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @return A 32 bit integer.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     * @throw HdfsTimeout
+     */
+    virtual int32_t readVarint32(int timeout) = 0;
+
+    /**
+     * Test if the socket can be read without blocking.
+     * @param timeout Time timeout interval of this operation, negative means infinite.
+     * @return Return true if the socket can be read without blocking, false on timeout.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual bool poll(int timeout) = 0;
+
+};
+
+/**
+ * An implement of BufferedSocketReader.
+ */
+class BufferedSocketReaderImpl: public BufferedSocketReader {
+public:
+    BufferedSocketReaderImpl(Socket & s);
+
+    int32_t read(char * b, int32_t s);
+
+    void readFully(char * b, int32_t s, int timeout);
+
+    int32_t readBigEndianInt32(int timeout);
+
+    int32_t readVarint32(int timeout);
+
+    bool poll(int timeout);
+
+private:
+    //for test
+    BufferedSocketReaderImpl(Socket & s, const std::vector<char> & buffer) :
+        cursor(0), size(buffer.size()), sock(s), buffer(buffer) {
+    }
+
+private:
+    int32_t cursor;
+    int32_t size;
+    Socket & sock;
+    std::vector<char> buffer;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_NETWORK_BUFFEREDSOCKET_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Socket.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Socket.h
new file mode 100644
index 0000000..43968dc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Socket.h
@@ -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.
+ */
+
+#ifndef _HDFS_LIBHDFS3_NETWORK_SOCKET_H_
+#define _HDFS_LIBHDFS3_NETWORK_SOCKET_H_
+
+#include <netdb.h>
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class Socket {
+public:
+
+    virtual ~Socket() {
+    }
+
+    /**
+     * Read data from socket.
+     * If there is nothing can be read, the caller will be blocked.
+     * @param buffer The buffer to store the data.
+     * @param size The size of bytes to be read.
+     * @return The size of data already read.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     */
+    virtual int32_t read(char * buffer, int32_t size) = 0;
+
+    /**
+     * Read data from socket until get enough data.
+     * If there is not enough data can be read, the caller will be blocked.
+     * @param buffer The buffer to store the data.
+     * @param size The size of bytes to be read.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     * @throw HdfsTimeout
+     */
+    virtual void readFully(char * buffer, int32_t size, int timeout) = 0;
+
+    /**
+     * Send data to socket.
+     * The caller will be blocked until send operation finished,
+     *      but not guarantee that all data has been sent.
+     * @param buffer The data to be sent.
+     * @param size The size of bytes to be sent.
+     * @return The size of data already be sent.
+     * @throw HdfsNetworkException
+     */
+    virtual int32_t write(const char * buffer, int32_t size) = 0;
+
+    /**
+     * Send all data to socket.
+     * The caller will be blocked until all data has been sent.
+     * @param buffer The data to be sent.
+     * @param size The size of bytes to be sent.
+     * @param timeout The timeout interval of this write operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual void writeFully(const char * buffer, int32_t size, int timeout) = 0;
+
+    /**
+     * Connection to a tcp server.
+     * @param host The host of server.
+     * @param port The port of server.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual void connect(const char * host, int port, int timeout) = 0;
+
+    /**
+     * Connection to a tcp server.
+     * @param host The host of server.
+     * @param port The port of server.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual void connect(const char * host, const char * port, int timeout) = 0;
+
+    /**
+     * Connection to a tcp server.
+     * @param paddr The address of server.
+     * @param host The host of server used in error message.
+     * @param port The port of server used in error message.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual void connect(struct addrinfo * paddr, const char * host,
+                         const char * port, int timeout) = 0;
+
+    /**
+     * Test if the socket can be read or written without blocking.
+     * @param read Test socket if it can be read.
+     * @param write Test socket if it can be written.
+     * @param timeout Time timeout interval of this operation, negative means infinite.
+     * @return Return true if the socket can be read or written without blocking, false on timeout.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    virtual bool poll(bool read, bool write, int timeout) = 0;
+
+    /**
+     * Set socket no delay mode.
+     * @param enable If true, set socket into no delay mode, else delay mode.
+     * @throw HdfsNetworkException
+     */
+    virtual void setNoDelay(bool enable) = 0;
+
+    /**
+     * Set socket blocking mode.
+     * @param enable If true, set socket into blocking mode, else non-block mode.
+     * @throw HdfsNetworkException
+     */
+    virtual void setBlockMode(bool enable) = 0;
+
+    /**
+     * Set socket linger timeout
+     * @param timeout Linger timeout of the socket in millisecond, disable linger if it is less than 0.
+     * @throw HdfsNetworkException
+     */
+    virtual void setLingerTimeout(int timeout) = 0;
+
+    /**
+     * Shutdown and close the socket.
+     * @throw nothrow
+     */
+    virtual void close() = 0;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_NETWORK_SOCKET_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Syscall.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Syscall.h
new file mode 100644
index 0000000..5dab57c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Syscall.h
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_NETWORK_SYSCALL_H_
+#define _HDFS_LIBHDFS3_NETWORK_SYSCALL_H_
+
+#include <fcntl.h>
+#include <netdb.h>
+#include <poll.h>
+#include <sys/socket.h>
+#include <unistd.h>
+
+namespace real_syscalls {
+
+using ::recv;
+using ::send;
+using ::getaddrinfo;
+using ::freeaddrinfo;
+using ::socket;
+using ::connect;
+using ::getpeername;
+using ::fcntl;
+using ::setsockopt;
+using ::poll;
+using ::shutdown;
+using ::close;
+
+}
+
+#ifdef MOCK
+
+#include "MockSystem.h"
+namespace syscalls = mock_systems;
+
+#else
+
+namespace syscalls = real_syscalls;
+
+#endif
+
+#endif /* _HDFS_LIBHDFS3_NETWORK_SYSCALL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.cc
new file mode 100644
index 0000000..de2db9d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.cc
@@ -0,0 +1,406 @@
+/**
+ * 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 "platform.h"
+
+#include <arpa/inet.h>
+#include <cassert>
+#include <climits>
+#include <cstring>
+#include <errno.h>
+#include <fcntl.h>
+#include <netdb.h>
+#include <netinet/in.h>
+#include <netinet/tcp.h>
+#include <poll.h>
+#include <stdint.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include <sstream>
+
+#include "DateTime.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "TcpSocket.h"
+#include "Syscall.h"
+
+// Linux defines a constant that you can use inside send() to prevent SIGPIPE
+// from being raised.  When this constant is present, we want to use it.  When
+// it is not present, we just pass 0 (no flag).
+#ifndef MSG_NOSIGNAL
+#define MSG_NOSIGNAL 0
+#endif
+
+namespace hdfs {
+namespace internal {
+
+// MacOS and some other BSD-based operating systems allow you to set
+// SO_NOSIGPIPE on a socket to prevent writes to that socket from raising
+// SIGPIPE.
+void TcpSocketImpl::setNoSigPipe() {
+#ifdef SO_NOSIGPIPE
+    int flag = 1;
+
+    if (syscalls::setsockopt(sock, SOL_SOCKET, SO_NOSIGPIPE, (char *) &flag,
+                               sizeof(flag))) {
+        THROW(HdfsNetworkException, "Set socket flag failed for remote "
+              "node %s: %s", remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+#endif
+}
+
+TcpSocketImpl::TcpSocketImpl() :
+    sock(-1), lingerTimeout(-1) {
+}
+
+TcpSocketImpl::~TcpSocketImpl() {
+    close();
+}
+
+int32_t TcpSocketImpl::read(char * buffer, int32_t size) {
+    assert(-1 != sock);
+    assert(NULL != buffer && size > 0);
+    int32_t rc;
+
+    do {
+        rc = syscalls::recv(sock, buffer, size, 0);
+    } while (-1 == rc && EINTR == errno && !CheckOperationCanceled());
+
+    if (-1 == rc) {
+        THROW(HdfsNetworkException, "Read %d bytes failed from %s: %s",
+              size, remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    if (0 == rc) {
+        THROW(HdfsEndOfStream, "Read %d bytes failed from %s: End of the stream", size, remoteAddr.c_str());
+    }
+
+    return rc;
+}
+
+void TcpSocketImpl::readFully(char * buffer, int32_t size, int timeout) {
+    assert(-1 != sock);
+    assert(NULL != buffer && size > 0);
+    int32_t todo = size, rc;
+    int deadline = timeout;
+
+    while (todo > 0) {
+        steady_clock::time_point s = steady_clock::now();
+        CheckOperationCanceled();
+
+        if (poll(true, false, deadline)) {
+            rc = read(buffer + (size - todo), todo);
+            todo -= rc;
+        }
+
+        steady_clock::time_point e = steady_clock::now();
+
+        if (timeout > 0) {
+            deadline -= ToMilliSeconds(s, e);
+        }
+
+        if (todo > 0 && timeout >= 0 && deadline <= 0) {
+            THROW(HdfsTimeoutException, "Read %d bytes timeout from %s", size, remoteAddr.c_str());
+        }
+    }
+}
+
+int32_t TcpSocketImpl::write(const char * buffer, int32_t size) {
+    assert(-1 != sock);
+    assert(NULL != buffer && size > 0);
+    int32_t rc;
+
+    do {
+        rc = syscalls::send(sock, buffer, size, MSG_NOSIGNAL);
+    } while (-1 == rc && EINTR == errno && !CheckOperationCanceled());
+
+    if (-1 == rc) {
+        THROW(HdfsNetworkException, "Write %d bytes failed to %s: %s",
+              size, remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    return rc;
+}
+
+void TcpSocketImpl::writeFully(const char * buffer, int32_t size, int timeout) {
+    assert(-1 != sock);
+    assert(NULL != buffer && size > 0);
+    int32_t todo = size, rc;
+    int deadline = timeout;
+
+    while (todo > 0) {
+        steady_clock::time_point s = steady_clock::now();
+        CheckOperationCanceled();
+
+        if (poll(false, true, deadline)) {
+            rc = write(buffer + (size - todo), todo);
+            todo -= rc;
+        }
+
+        steady_clock::time_point e = steady_clock::now();
+
+        if (timeout > 0) {
+            deadline -= ToMilliSeconds(s, e);
+        }
+
+        if (todo > 0 && timeout >= 0 && deadline <= 0) {
+            THROW(HdfsTimeoutException, "Write %d bytes timeout to %s",
+                  size, remoteAddr.c_str());
+        }
+    }
+}
+
+void TcpSocketImpl::connect(const char * host, int port, int timeout) {
+    std::stringstream ss;
+    ss << port;
+    connect(host, ss.str().c_str(), timeout);
+}
+
+void TcpSocketImpl::connect(const char * host, const char * port, int timeout) {
+    assert(-1 == sock);
+    struct addrinfo hints, *addrs, *paddr;
+    memset(&hints, 0, sizeof(hints));
+    hints.ai_family = PF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    int retval = syscalls::getaddrinfo(host, port, &hints, &addrs);
+
+    if (0 != retval) {
+        THROW(HdfsNetworkConnectException, "Failed to resolve address \"%s:%s\" %s",
+              host, port, gai_strerror(retval));
+    }
+
+    int deadline = timeout;
+    std::stringstream ss;
+    ss << "\"" << host << ":" << port << "\"";
+    remoteAddr = ss.str();
+
+    try {
+        for (paddr = addrs; NULL != paddr; paddr = paddr->ai_next) {
+            steady_clock::time_point s = steady_clock::now();
+            CheckOperationCanceled();
+
+            try {
+                connect(paddr, host, port, deadline);
+            } catch (HdfsNetworkConnectException & e) {
+                if (NULL == paddr->ai_next) {
+                    throw;
+                }
+            } catch (HdfsTimeoutException & e) {
+                if (NULL == paddr->ai_next) {
+                    throw;
+                }
+            }
+
+            if (-1 != sock) {
+                syscalls::freeaddrinfo(addrs);
+                return;
+            }
+
+            steady_clock::time_point e = steady_clock::now();
+
+            if (timeout > 0) {
+                deadline -= ToMilliSeconds(s, e);
+            }
+
+            if (-1 == sock && timeout >= 0 && deadline <= 0) {
+                THROW(HdfsTimeoutException, "Connect to \"%s:%s\" timeout", host, port);
+            }
+        }
+    } catch (...) {
+        syscalls::freeaddrinfo(addrs);
+        throw;
+    }
+}
+
+void TcpSocketImpl::connect(struct addrinfo * paddr, const char * host,
+                            const char * port, int timeout) {
+    assert(-1 == sock);
+    sock = syscalls::socket(paddr->ai_family, paddr->ai_socktype,
+                              paddr->ai_protocol);
+
+    if (-1 == sock) {
+        THROW(HdfsNetworkException,
+              "Create socket failed when connect to %s: %s",
+              remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    if (lingerTimeout >= 0) {
+        setLingerTimeoutInternal(lingerTimeout);
+    }
+
+#ifdef __linux__
+    /*
+     * on linux some kernel use SO_SNDTIMEO as connect timeout.
+     * It is OK to set a very large value here since the user has its own timeout mechanism.
+     */
+    setSendTimeout(3600000);
+#endif
+
+    try {
+        setBlockMode(false);
+        setNoSigPipe();
+
+        int rc = 0;
+        do {
+            rc = syscalls::connect(sock, paddr->ai_addr, paddr->ai_addrlen);
+        } while (rc < 0 && EINTR == errno && !CheckOperationCanceled());
+
+        if (rc < 0) {
+            if (EINPROGRESS != errno && EWOULDBLOCK != errno) {
+                if (ETIMEDOUT == errno) {
+                    THROW(HdfsTimeoutException, "Connect to \"%s:%s\" timeout",
+                          host, port);
+                } else {
+                    THROW(HdfsNetworkConnectException,
+                          "Connect to \"%s:%s\" failed: %s",
+                          host, port, GetSystemErrorInfo(errno));
+                }
+            }
+
+            if (!poll(false, true, timeout)) {
+                THROW(HdfsTimeoutException, "Connect to \"%s:%s\" timeout", host, port);
+            }
+
+            struct sockaddr peer;
+
+            unsigned int len = sizeof(peer);
+
+            memset(&peer, 0, sizeof(peer));
+
+            if (syscalls::getpeername(sock, &peer, &len)) {
+                /*
+                 * connect failed, find out the error info.
+                 */
+                char c;
+                rc = syscalls::recv(sock, &c, 1, 0);
+                assert(rc < 0);
+
+                if (ETIMEDOUT == errno) {
+                    THROW(HdfsTimeoutException, "Connect to \"%s:%s\" timeout",
+                          host, port);
+                }
+
+                THROW(HdfsNetworkConnectException, "Connect to \"%s:%s\" failed: %s",
+                      host, port, GetSystemErrorInfo(errno));
+            }
+        }
+
+        setBlockMode(true);
+    } catch (...) {
+        close();
+        throw;
+    }
+}
+
+void TcpSocketImpl::setBlockMode(bool enable) {
+    int flag;
+    flag = syscalls::fcntl(sock, F_GETFL, 0);
+
+    if (-1 == flag) {
+        THROW(HdfsNetworkException, "Get socket flag failed for remote node %s: %s",
+              remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    flag = enable ? (flag & ~O_NONBLOCK) : (flag | O_NONBLOCK);
+
+    if (-1 == syscalls::fcntl(sock, F_SETFL, flag)) {
+        THROW(HdfsNetworkException, "Set socket flag failed for remote "
+              "node %s: %s", remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+}
+
+bool TcpSocketImpl::poll(bool read, bool write, int timeout) {
+    assert(-1 != sock);
+    int rc;
+    struct pollfd pfd;
+
+    do {
+        memset(&pfd, 0, sizeof(pfd));
+        pfd.fd = sock;
+
+        if (read) {
+            pfd.events |= POLLIN;
+        }
+
+        if (write) {
+            pfd.events |= POLLOUT;
+        }
+
+        rc = syscalls::poll(&pfd, 1, timeout);
+    } while (-1 == rc && EINTR == errno && !CheckOperationCanceled());
+
+    if (-1 == rc) {
+        THROW(HdfsNetworkException, "Poll failed for remote node %s: %s",
+              remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+
+    return 0 != rc;
+}
+
+void TcpSocketImpl::setNoDelay(bool enable) {
+    assert(-1 != sock);
+    int flag = enable ? 1 : 0;
+
+    if (syscalls::setsockopt(sock, IPPROTO_TCP, TCP_NODELAY, (char *) &flag,
+                               sizeof(flag))) {
+        THROW(HdfsNetworkException, "Set socket flag failed for remote "
+              "node %s: %s", remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+}
+
+void TcpSocketImpl::setLingerTimeout(int timeout) {
+    lingerTimeout = timeout;
+}
+
+void TcpSocketImpl::setLingerTimeoutInternal(int timeout) {
+    assert(-1 != sock);
+    struct linger l;
+    l.l_onoff = timeout > 0 ? true : false;
+    l.l_linger = timeout > 0 ? timeout : 0;
+
+    if (syscalls::setsockopt(sock, SOL_SOCKET, SO_LINGER, &l, sizeof(l))) {
+        THROW(HdfsNetworkException, "Set socket flag failed for remote "
+              "node %s: %s", remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+}
+
+void TcpSocketImpl::setSendTimeout(int timeout) {
+    assert(-1 != sock);
+    struct timeval timeo;
+    timeo.tv_sec = timeout / 1000;
+    timeo.tv_usec = (timeout % 1000) * 1000;
+
+    if (syscalls::setsockopt(sock, SOL_SOCKET, SO_SNDTIMEO,
+                                &timeo, sizeof(timeo))) {
+        THROW(HdfsNetworkException, "Set socket flag failed for remote "
+              "node %s: %s", remoteAddr.c_str(), GetSystemErrorInfo(errno));
+    }
+}
+
+void TcpSocketImpl::close() {
+    if (-1 != sock) {
+        syscalls::shutdown(sock, SHUT_RDWR);
+        syscalls::close(sock);
+        sock = -1;
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.h
new file mode 100644
index 0000000..ff90b20
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.h
@@ -0,0 +1,172 @@
+/**
+ * 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_NETWORK_TCPSOCKET_H_
+#define _HDFS_LIBHDFS3_NETWORK_TCPSOCKET_H_
+
+#include "Socket.h"
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * A tcp socket client
+ */
+class TcpSocketImpl: public Socket {
+public:
+    /**
+     * Construct a Socket object.
+     * @throw nothrow
+     */
+    TcpSocketImpl();
+
+    /**
+     * Destroy a TcpSocketImpl instance.
+     */
+    ~TcpSocketImpl();
+
+    /**
+     * Read data from socket.
+     * If there is nothing can be read, the caller will be blocked.
+     * @param buffer The buffer to store the data.
+     * @param size The size of bytes to be read.
+     * @return The size of data already read.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     */
+    int32_t read(char * buffer, int32_t size);
+
+    /**
+     * Read data from socket until get enough data.
+     * If there is not enough data can be read, the caller will be blocked.
+     * @param buffer The buffer to store the data.
+     * @param size The size of bytes to be read.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsEndOfStream
+     * @throw HdfsTimeout
+     */
+    void readFully(char * buffer, int32_t size, int timeout);
+
+    /**
+     * Send data to socket.
+     * The caller will be blocked until send operation finished,
+     *      but not guarantee that all data has been sent.
+     * @param buffer The data to be sent.
+     * @param size The size of bytes to be sent.
+     * @return The size of data already be sent.
+     * @throw HdfsNetworkException
+     */
+    int32_t write(const char * buffer, int32_t size);
+
+    /**
+     * Send all data to socket.
+     * The caller will be blocked until all data has been sent.
+     * @param buffer The data to be sent.
+     * @param size The size of bytes to be sent.
+     * @param timeout The timeout interval of this write operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    void writeFully(const char * buffer, int32_t size, int timeout);
+
+    /**
+     * Connection to a tcp server.
+     * @param host The host of server.
+     * @param port The port of server.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    void connect(const char * host, int port, int timeout);
+
+    /**
+     * Connection to a tcp server.
+     * @param host The host of server.
+     * @param port The port of server.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    void connect(const char * host, const char * port, int timeout);
+
+    /**
+     * Connection to a tcp server.
+     * @param paddr The address of server.
+     * @param host The host of server used in error message.
+     * @param port The port of server used in error message.
+     * @param timeout The timeout interval of this read operation, negative means infinite.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    void connect(struct addrinfo * paddr, const char * host, const char * port,
+                 int timeout);
+
+    /**
+     * Test if the socket can be read or written without blocking.
+     * @param read Test socket if it can be read.
+     * @param write Test socket if it can be written.
+     * @param timeout Time timeout interval of this operation, negative means infinite.
+     * @return Return true if the socket can be read or written without blocking, false on timeout.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    bool poll(bool read, bool write, int timeout);
+
+    /**
+     * Set socket no delay mode.
+     * @param enable If true, set socket into no delay mode, else delay mode.
+     * @throw HdfsNetworkException
+     */
+    void setNoDelay(bool enable);
+
+    /**
+     * Set socket blocking mode.
+     * @param enable If true, set socket into blocking mode, else non-block mode.
+     * @throw HdfsNetworkException
+     */
+    void setBlockMode(bool enable);
+
+    /**
+     * Set socket linger timeout
+     * @param timeout Linger timeout of the socket in millisecond, disable linger if it is less than 0.
+     * @throw HdfsNetworkException
+     */
+    void setLingerTimeout(int timeout);
+
+    /**
+     * Shutdown and close the socket.
+     * @throw nothrow
+     */
+    void close();
+
+private:
+    void setNoSigPipe();
+    void setLingerTimeoutInternal(int timeout);
+    void setSendTimeout(int timeout);
+
+private:
+    int sock;
+    int lingerTimeout;
+    std::string remoteAddr;  //used for error message
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_NETWORK_TCPSOCKET_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.cc
new file mode 100644
index 0000000..3f63ff6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.cc
@@ -0,0 +1,46 @@
+/**
+ * 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 "RpcAuth.h"
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+namespace hdfs {
+namespace internal {
+
+AuthMethod RpcAuth::ParseMethod(const std::string &str) {
+    if (0 == strcasecmp(str.c_str(), "SIMPLE")) {
+        return AuthMethod::SIMPLE;
+    } else if (0 == strcasecmp(str.c_str(), "KERBEROS")) {
+        return AuthMethod::KERBEROS;
+    } else if (0 == strcasecmp(str.c_str(), "TOKEN")) {
+        return AuthMethod::TOKEN;
+    } else {
+        THROW(InvalidParameter, "RpcAuth: Unknown auth mechanism type: %s",
+              str.c_str());
+    }
+}
+
+size_t RpcAuth::hash_value() const {
+    size_t values[] = { Int32Hasher(method), user.hash_value() };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.h
new file mode 100644
index 0000000..e82c28d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.h
@@ -0,0 +1,93 @@
+/**
+ * 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_RPC_RPCAUTH_H_
+#define _HDFS_LIBHDFS3_RPC_RPCAUTH_H_
+
+#include "client/UserInfo.h"
+#include "Hash.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+enum AuthMethod {
+    SIMPLE = 80, KERBEROS = 81, //"GSSAPI"
+    TOKEN = 82, //"DIGEST-MD5"
+    UNKNOWN = 255
+};
+
+enum AuthProtocol {
+    NONE = 0, SASL = -33
+};
+
+class RpcAuth {
+public:
+    RpcAuth() :
+        method(SIMPLE) {
+    }
+
+    explicit RpcAuth(AuthMethod mech) :
+        method(mech) {
+    }
+
+    RpcAuth(const UserInfo &ui, AuthMethod mech) :
+        method(mech), user(ui) {
+    }
+
+    AuthProtocol getProtocol() const {
+        return method == SIMPLE ? AuthProtocol::NONE : AuthProtocol::SASL;
+    }
+
+    const UserInfo &getUser() const {
+        return user;
+    }
+
+    UserInfo &getUser() {
+        return user;
+    }
+
+    void setUser(const UserInfo &user) {
+        this->user = user;
+    }
+
+    AuthMethod getMethod() const {
+        return method;
+    }
+
+    size_t hash_value() const;
+
+    bool operator ==(const RpcAuth &other) const {
+        return method == other.method && user == other.user;
+    }
+
+public:
+    static AuthMethod ParseMethod(const std::string &str);
+
+private:
+    AuthMethod method;
+    UserInfo user;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::RpcAuth);
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCAUTH_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcCall.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcCall.h
new file mode 100644
index 0000000..7c6e316
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcCall.h
@@ -0,0 +1,78 @@
+/**
+ * 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_RPC_RPCCALL_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCALL_H_
+
+#include "google/protobuf/message.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class RpcCall {
+public:
+    RpcCall(bool idemp, std::string n, google::protobuf::Message *req,
+            google::protobuf::Message *resp) :
+        idempotent(idemp), name(n), request(req), response(resp) {
+    }
+
+    bool isIdempotent() const {
+        return idempotent;
+    }
+
+    const char *getName() const {
+        return name.c_str();
+    }
+
+    void setIdempotent(bool idempotent) {
+        this->idempotent = idempotent;
+    }
+
+    void setName(const std::string &name) {
+        this->name = name;
+    }
+
+    google::protobuf::Message *getRequest() {
+        return request;
+    }
+
+    void setRequest(google::protobuf::Message *request) {
+        this->request = request;
+    }
+
+    google::protobuf::Message *getResponse() {
+        return response;
+    }
+
+    void setResponse(google::protobuf::Message *response) {
+        this->response = response;
+    }
+
+private:
+    bool idempotent;
+    std::string name;
+    google::protobuf::Message *request;
+    google::protobuf::Message *response;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCALL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.cc
new file mode 100644
index 0000000..6057a01
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.cc
@@ -0,0 +1,876 @@
+/**
+ * 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 "IpcConnectionContext.pb.h"
+#include "Logger.h"
+#include "RpcChannel.h"
+#include "RpcClient.h"
+#include "RpcContentWrapper.h"
+#include "RpcHeader.pb.h"
+#include "RpcHeader.pb.h"
+#include "Thread.h"
+#include "WriteBuffer.h"
+#include "server/RpcHelper.h"
+
+#include <google/protobuf/io/coded_stream.h>
+
+#define RPC_HEADER_MAGIC "hrpc"
+#define RPC_HEADER_VERSION 9
+#define SERIALIZATION_TYPE_PROTOBUF 0
+#define CONNECTION_CONTEXT_CALL_ID -3
+
+using namespace ::google::protobuf;
+using namespace google::protobuf::io;
+using namespace hadoop::common;
+using namespace hadoop::hdfs;
+
+namespace hdfs {
+namespace internal {
+
+RpcChannelImpl::RpcChannelImpl(const RpcChannelKey &k, RpcClient &c) :
+    refs(0), available(false), key(k), client(c) {
+    sock = shared_ptr<Socket>(new TcpSocketImpl);
+    sock->setLingerTimeout(k.getConf().getLingerTimeout());
+    in = shared_ptr<BufferedSocketReader>(
+             new BufferedSocketReaderImpl(
+                 *static_cast<TcpSocketImpl *>(sock.get())));
+    lastActivity = lastIdle = steady_clock::now();
+}
+
+RpcChannelImpl::RpcChannelImpl(const RpcChannelKey &k, Socket *s,
+                               BufferedSocketReader *in, RpcClient &c) :
+    refs(0), available(false), key(k), client(c) {
+    sock = shared_ptr<Socket>(s);
+    this->in = shared_ptr<BufferedSocketReader>(in);
+    lastActivity = lastIdle = steady_clock::now();
+}
+
+RpcChannelImpl::~RpcChannelImpl() {
+    assert(pendingCalls.empty());
+    assert(refs == 0);
+
+    if (available) {
+        sock->close();
+    }
+}
+
+void RpcChannelImpl::close(bool immediate) {
+    lock_guard<mutex> lock(writeMut);
+    --refs;
+    assert(refs >= 0);
+
+    if (immediate && !refs) {
+        assert(pendingCalls.empty());
+        available = false;
+        sock->close();
+    }
+}
+
+void RpcChannelImpl::sendSaslMessage(RpcSaslProto *msg, Message *resp) {
+    int totalLen;
+    WriteBuffer buffer;
+    RpcRequestHeaderProto rpcHeader;
+    rpcHeader.set_callid(AuthProtocol::SASL);
+    rpcHeader.set_clientid(client.getClientId());
+    rpcHeader.set_retrycount(INVALID_RETRY_COUNT);
+    rpcHeader.set_rpckind(RPC_PROTOCOL_BUFFER);
+    rpcHeader.set_rpcop(RpcRequestHeaderProto_OperationProto_RPC_FINAL_PACKET);
+    RpcContentWrapper wrapper(&rpcHeader, msg);
+    totalLen = wrapper.getLength();
+    buffer.writeBigEndian(totalLen);
+    wrapper.writeTo(buffer);
+    sock->writeFully(buffer.getBuffer(0), buffer.getDataSize(0),
+                     key.getConf().getWriteTimeout());
+    RpcRemoteCallPtr call(
+        new RpcRemoteCall(RpcCall(false, "sasl message", NULL, resp),
+                          AuthProtocol::SASL, client.getClientId()));
+    pendingCalls[AuthProtocol::SASL] = call;
+}
+
+const RpcSaslProto_SaslAuth * RpcChannelImpl::createSaslClient(
+    const RepeatedPtrField<RpcSaslProto_SaslAuth> *auths) {
+    const RpcSaslProto_SaslAuth *auth = NULL;
+    Token token;
+
+    for (int i = 0; i < auths->size(); ++i) {
+        auth = &auths->Get(i);
+        RpcAuth method(RpcAuth::ParseMethod(auth->method()));
+
+        if (method.getMethod() == AuthMethod::TOKEN && key.hasToken()) {
+            token = key.getToken();
+            break;
+        } else if (method.getMethod() == AuthMethod::KERBEROS) {
+            break;
+        } else if (method.getMethod() == AuthMethod::SIMPLE) {
+            return auth;
+        } else if (method.getMethod() == AuthMethod::UNKNOWN) {
+            return auth;
+        } else {
+            auth = NULL;
+        }
+    }
+
+    if (!auth) {
+        std::stringstream ss;
+        ss << "Client cannot authenticate via: ";
+
+        for (int i = 0; i < auths->size(); ++i) {
+            auth = &auths->Get(i);
+            ss << auth->mechanism() << ", ";
+        }
+
+        THROW(AccessControlException, "%s", ss.str().c_str());
+    }
+
+    saslClient = shared_ptr<SaslClient>(
+                     new SaslClient(*auth, token,
+                        key.getAuth().getUser().getPrincipal()));
+    return auth;
+}
+
+std::string RpcChannelImpl::saslEvaluateToken(RpcSaslProto &response,
+                                              bool serverIsDone) {
+    std::string token;
+
+    if (response.has_token()) {
+        token = saslClient->evaluateChallenge(response.token());
+    } else if (!serverIsDone) {
+        THROW(AccessControlException, "Server challenge contains no token");
+    }
+
+    if (serverIsDone) {
+        if (!saslClient->isComplete()) {
+            THROW(AccessControlException, "Client is out of sync with server");
+        }
+
+        if (!token.empty()) {
+            THROW(AccessControlException, "Client generated spurious "
+                  "response");
+        }
+    }
+
+    return token;
+}
+
+RpcAuth RpcChannelImpl::setupSaslConnection() {
+    RpcAuth retval;
+    RpcSaslProto negotiateRequest, response, msg;
+    negotiateRequest.set_state(RpcSaslProto_SaslState_NEGOTIATE);
+    sendSaslMessage(&negotiateRequest, &response);
+    bool done = false;
+
+    do {
+        readOneResponse(false);
+        msg.Clear();
+
+        switch (response.state()) {
+        case RpcSaslProto_SaslState_NEGOTIATE: {
+            const RpcSaslProto_SaslAuth *auth = createSaslClient(
+                    &response.auths());
+            retval = RpcAuth(RpcAuth::ParseMethod(auth->method()));
+
+            if (retval.getMethod() == AuthMethod::SIMPLE) {
+                done = true;
+            } else if (retval.getMethod() == AuthMethod::UNKNOWN) {
+                THROW(AccessControlException, "Unknown auth mechanism");
+            } else {
+                std::string respToken;
+                RpcSaslProto_SaslAuth *respAuth = msg.add_auths();
+                respAuth->CopyFrom(*auth);
+                std::string chanllege;
+
+                if (auth->has_challenge()) {
+                    chanllege = auth->challenge();
+                    respAuth->clear_challenge();
+                }
+
+                respToken = saslClient->evaluateChallenge(chanllege);
+
+                if (!respToken.empty()) {
+                    msg.set_token(respToken);
+                }
+
+                msg.set_state(RpcSaslProto_SaslState_INITIATE);
+            }
+
+            break;
+        }
+
+        case RpcSaslProto_SaslState_CHALLENGE: {
+            if (!saslClient) {
+                THROW(AccessControlException, "Server sent unsolicited challenge");
+            }
+
+            std::string token = saslEvaluateToken(response, false);
+            msg.set_token(token);
+            msg.set_state(RpcSaslProto_SaslState_RESPONSE);
+            break;
+        }
+
+        case RpcSaslProto_SaslState_SUCCESS:
+            if (!saslClient) {
+                retval = RpcAuth(AuthMethod::SIMPLE);
+            } else {
+                saslEvaluateToken(response, true);
+            }
+
+            done = true;
+            break;
+
+        default:
+            break;
+        }
+
+        if (!done) {
+            response.Clear();
+            sendSaslMessage(&msg, &response);
+        }
+    } while (!done);
+
+    return retval;
+}
+
+void RpcChannelImpl::connect() {
+    int sleep = 1;
+    exception_ptr lastError;
+    const RpcConfig & conf = key.getConf();
+    const RpcServerInfo & server = key.getServer();
+
+    for (int i = 0; i < conf.getMaxRetryOnConnect(); ++i) {
+        RpcAuth auth = key.getAuth();
+
+        try {
+            while (true) {
+                sock->connect(server.getHost().c_str(),
+                              server.getPort().c_str(),
+                              conf.getConnectTimeout());
+                sock->setNoDelay(conf.isTcpNoDelay());
+                sendConnectionHeader();
+
+                if (auth.getProtocol() == AuthProtocol::SASL) {
+                    auth = setupSaslConnection();
+                    if (auth.getProtocol() == AuthProtocol::SASL) {
+                        //success
+                        break;
+                    }
+
+                    /*
+                     * switch to other auth protocol
+                     */
+                    sock->close();
+                    CheckOperationCanceled();
+                } else {
+                    break;
+                }
+            }
+
+            auth.setUser(key.getAuth().getUser());
+            sendConnectionContent(auth);
+            available = true;
+            lastActivity = lastIdle = steady_clock::now();
+            return;
+        } catch (const SaslException & e) {
+            /*
+             * Namenode may treat this connect as replay, retry later
+             */
+            sleep = (rand() % 5) + 1;
+            lastError = current_exception();
+            LOG(LOG_ERROR,
+                "Failed to setup RPC connection to \"%s:%s\" caused by:\n%s",
+                server.getHost().c_str(), server.getPort().c_str(),
+                GetExceptionDetail(e));
+        } catch (const HdfsNetworkException & e) {
+            sleep = 1;
+            lastError = current_exception();
+            LOG(LOG_ERROR,
+                "Failed to setup RPC connection to \"%s:%s\" caused by:\n%s",
+                server.getHost().c_str(), server.getPort().c_str(),
+                GetExceptionDetail(e));
+        } catch (const HdfsTimeoutException & e) {
+            sleep = 1;
+            lastError = current_exception();
+            LOG(LOG_ERROR,
+                "Failed to setup RPC connection to \"%s:%s\" caused by:\n%s",
+                server.getHost().c_str(), server.getPort().c_str(),
+                GetExceptionDetail(e));
+        }
+
+        if (i + 1 < conf.getMaxRetryOnConnect()) {
+            LOG(INFO,
+                "Retrying connect to server: \"%s:%s\". "
+                "Already tried %d time(s)", server.getHost().c_str(),
+                server.getPort().c_str(), i + 1);
+        }
+
+        sock->close();
+        CheckOperationCanceled();
+        sleep_for(seconds(sleep));
+    }
+
+    rethrow_exception(lastError);
+}
+
+exception_ptr RpcChannelImpl::invokeInternal(RpcRemoteCallPtr remote) {
+    const RpcCall & call = remote->getCall();
+    exception_ptr lastError;
+
+    try {
+        if (client.isRunning()) {
+            lock_guard<mutex> lock(writeMut);
+
+            if (!available) {
+                connect();
+            }
+
+            sendRequest(remote);
+        }
+
+        /*
+         * We use one call thread to check response,
+         * other thread will wait on RPC call complete.
+         */
+        while (client.isRunning()) {
+            if (remote->finished()) {
+                /*
+                 * Current RPC call has finished.
+                 * Wake up another thread to check response.
+                 */
+                wakeupOneCaller(remote->getIdentity());
+                break;
+            }
+
+            unique_lock<mutex> lock(readMut, defer_lock_t());
+
+            if (lock.try_lock()) {
+                /*
+                 * Current thread will check response.
+                 */
+                checkOneResponse();
+            } else {
+                /*
+                 * Another thread checks response, just wait.
+                 */
+                remote->wait();
+            }
+        }
+    } catch (const HdfsNetworkConnectException & e) {
+        try {
+            NESTED_THROW(HdfsFailoverException,
+                       "Failed to invoke RPC call \"%s\" on server \"%s:%s\"",
+                       call.getName(), key.getServer().getHost().c_str(),
+                       key.getServer().getPort().c_str());
+        } catch (const HdfsFailoverException & e) {
+            lastError = current_exception();
+        }
+    } catch (const HdfsNetworkException & e) {
+        try {
+            NESTED_THROW(HdfsRpcException,
+                       "Failed to invoke RPC call \"%s\" on server \"%s:%s\"",
+                       call.getName(), key.getServer().getHost().c_str(),
+                       key.getServer().getPort().c_str());
+        } catch (const HdfsRpcException & e) {
+            lastError = current_exception();
+        }
+    } catch (const HdfsTimeoutException & e) {
+        try {
+            NESTED_THROW(HdfsFailoverException,
+                       "Failed to invoke RPC call \"%s\" on server \"%s:%s\"",
+                       call.getName(), key.getServer().getHost().c_str(),
+                       key.getServer().getPort().c_str());
+        } catch (const HdfsFailoverException & e) {
+            lastError = current_exception();
+        }
+    } catch (const HdfsRpcException & e) {
+        lastError = current_exception();
+    } catch (const HdfsIOException & e) {
+        try {
+            NESTED_THROW(HdfsRpcException,
+                       "Failed to invoke RPC call \"%s\" on server \"%s:%s\"",
+                       call.getName(), key.getServer().getHost().c_str(),
+                       key.getServer().getPort().c_str());
+        } catch (const HdfsRpcException & e) {
+            lastError = current_exception();
+        }
+    }
+
+    return lastError;
+}
+
+void RpcChannelImpl::invoke(const RpcCall & call) {
+    assert(refs > 0);
+    RpcRemoteCallPtr remote;
+    exception_ptr lastError;
+
+    try {
+        bool retry = false;
+
+        do {
+            int32_t id = client.getCallId();
+            remote = RpcRemoteCallPtr(
+                  new RpcRemoteCall(call, id, client.getClientId()));
+            lastError = exception_ptr();
+            lastError = invokeInternal(remote);
+
+            if (lastError) {
+                lock_guard<mutex> lock(writeMut);
+                shutdown(lastError);
+
+                if (!retry && call.isIdempotent()) {
+                    retry = true;
+                    LOG(LOG_ERROR,
+                        "Failed to invoke RPC call \"%s\" on "
+                        "server \"%s:%s\": \n%s",
+                        call.getName(), key.getServer().getHost().c_str(),
+                        key.getServer().getPort().c_str(),
+                        GetExceptionDetail(lastError));
+                    LOG(INFO,
+                        "Retry idempotent RPC call \"%s\" on "
+                        "server \"%s:%s\"",
+                        call.getName(), key.getServer().getHost().c_str(),
+                        key.getServer().getPort().c_str());
+                } else {
+                    rethrow_exception(lastError);
+                }
+            } else {
+                break;
+            }
+        } while (retry);
+    } catch (const HdfsRpcServerException & e) {
+        if (!remote->finished()) {
+            /*
+             * a fatal error happened, the caller will unwrap it.
+             */
+            lock_guard<mutex> lock(writeMut);
+            lastError = current_exception();
+            shutdown(lastError);
+        }
+
+        /*
+         * else not a fatal error, check again at the end of this function.
+         */
+    } catch (const HdfsException & e) {
+        lock_guard<mutex> lock(writeMut);
+        lastError = current_exception();
+        shutdown(lastError);
+    }
+
+    /*
+     * if the call is not finished, either failed to setup connection,
+     * or client is closing.
+     */
+    if (!remote->finished() || !client.isRunning()) {
+        lock_guard<mutex> lock(writeMut);
+
+        if (lastError == exception_ptr()) {
+            try {
+                THROW(hdfs::HdfsRpcException,
+                      "Failed to invoke RPC call \"%s\", RPC channel "
+                      "to \"%s:%s\" is to be closed since RpcClient is "
+                      "closing", call.getName(),
+                      key.getServer().getHost().c_str(),
+                      key.getServer().getPort().c_str());
+            } catch (...) {
+                lastError = current_exception();
+            }
+        }
+
+        /*
+         * wake up all.
+         */
+        shutdown(lastError);
+        rethrow_exception(lastError);
+    }
+
+    remote->check();
+}
+
+void RpcChannelImpl::shutdown(exception_ptr reason) {
+    assert(reason != exception_ptr());
+    available = false;
+    cleanupPendingCalls(reason);
+    sock->close();
+}
+
+void RpcChannelImpl::wakeupOneCaller(int32_t id) {
+    lock_guard<mutex> lock(writeMut);
+    unordered_map<int32_t, RpcRemoteCallPtr>::iterator s, e;
+    e = pendingCalls.end();
+
+    for (s = pendingCalls.begin(); s != e; ++s) {
+        if (s->first != id) {
+            s->second->wakeup();
+            return;
+        }
+    }
+}
+
+void RpcChannelImpl::sendRequest(RpcRemoteCallPtr remote) {
+    WriteBuffer buffer;
+    assert(true == available);
+    remote->serialize(key.getProtocol(), buffer);
+    sock->writeFully(buffer.getBuffer(0), buffer.getDataSize(0),
+                     key.getConf().getWriteTimeout());
+    uint32_t id = remote->getIdentity();
+    pendingCalls[id] = remote;
+    lastActivity = lastIdle = steady_clock::now();
+}
+
+void RpcChannelImpl::cleanupPendingCalls(exception_ptr reason) {
+    assert(!writeMut.try_lock());
+    unordered_map<int32_t, RpcRemoteCallPtr>::iterator s, e;
+    e = pendingCalls.end();
+
+    for (s = pendingCalls.begin(); s != e; ++s) {
+        s->second->cancel(reason);
+    }
+
+    pendingCalls.clear();
+}
+
+void RpcChannelImpl::checkOneResponse() {
+    int ping = key.getConf().getPingTimeout();
+    int timeout = key.getConf().getRpcTimeout();
+    steady_clock::time_point start = steady_clock::now();
+
+    while (client.isRunning()) {
+        if (getResponse()) {
+            readOneResponse(true);
+            return;
+        } else {
+            if (ping > 0 && ToMilliSeconds(lastActivity,
+                    steady_clock::now()) >= ping) {
+                lock_guard<mutex> lock(writeMut);
+                sendPing();
+            }
+        }
+
+        if (timeout > 0 && ToMilliSeconds(start,
+                    steady_clock::now()) >= timeout) {
+            try {
+                THROW(hdfs::HdfsTimeoutException,
+                  "Timeout when wait for response from RPC channel \"%s:%s\"",
+                  key.getServer().getHost().c_str(),
+                  key.getServer().getPort().c_str());
+            } catch (...) {
+                NESTED_THROW(hdfs::HdfsRpcException,
+                  "Timeout when wait for response from RPC channel \"%s:%s\"",
+                  key.getServer().getHost().c_str(),
+                  key.getServer().getPort().c_str());
+            }
+        }
+    }
+}
+
+void RpcChannelImpl::sendPing() {
+    static const std::vector<char> pingRequest =
+      RpcRemoteCall::GetPingRequest(client.getClientId());
+
+    if (available) {
+        LOG(INFO,
+            "RPC channel to \"%s:%s\" got no response or idle for %d "
+            "milliseconds, sending ping.",
+            key.getServer().getHost().c_str(),
+            key.getServer().getPort().c_str(), key.getConf().getPingTimeout());
+        sock->writeFully(&pingRequest[0], pingRequest.size(),
+                         key.getConf().getWriteTimeout());
+        lastActivity = steady_clock::now();
+    }
+}
+
+bool RpcChannelImpl::checkIdle() {
+    unique_lock<mutex> lock(writeMut, defer_lock_t());
+
+    if (lock.try_lock()) {
+        if (!pendingCalls.empty() || refs > 0) {
+            lastIdle = steady_clock::now();
+            return false;
+        }
+
+        int idle = key.getConf().getMaxIdleTime();
+        int ping = key.getConf().getPingTimeout();
+
+        try {
+            //close the connection if idle timeout
+            if (ToMilliSeconds(lastIdle, steady_clock::now()) >= idle) {
+                sock->close();
+                return true;
+            }
+
+            //send ping
+            if (ping > 0 && ToMilliSeconds(lastActivity,
+                                           steady_clock::now()) >= ping) {
+                sendPing();
+            }
+        } catch (...) {
+            LOG(LOG_ERROR,
+                "Failed to send ping via idle RPC channel "
+                "to server \"%s:%s\": \n%s",
+                key.getServer().getHost().c_str(),
+                key.getServer().getPort().c_str(),
+                GetExceptionDetail(current_exception()));
+            sock->close();
+            return true;
+        }
+    }
+
+    return false;
+}
+
+void RpcChannelImpl::waitForExit() {
+    assert(!client.isRunning());
+
+    while (refs != 0) {
+        sleep_for(milliseconds(100));
+    }
+
+    assert(pendingCalls.empty());
+}
+
+/**
+ * Write the connection header - this is sent when connection is established
+ * +----------------------------------+
+ * |  "hrpc" 4 bytes                  |
+ * +----------------------------------+
+ * |  Version (1 byte)                |
+ * +----------------------------------+
+ * |  Service Class (1 byte)          |
+ * +----------------------------------+
+ * |  AuthProtocol (1 byte)           |
+ * +----------------------------------+
+ */
+void RpcChannelImpl::sendConnectionHeader() {
+    WriteBuffer buffer;
+    buffer.write(RPC_HEADER_MAGIC, strlen(RPC_HEADER_MAGIC));
+    buffer.write(static_cast<char>(RPC_HEADER_VERSION));
+    buffer.write(static_cast<char>(0));  //for future feature
+    buffer.write(static_cast<char>(key.getAuth().getProtocol()));
+    sock->writeFully(buffer.getBuffer(0), buffer.getDataSize(0),
+                     key.getConf().getWriteTimeout());
+}
+
+void RpcChannelImpl::buildConnectionContext(
+    IpcConnectionContextProto & connectionContext, const RpcAuth & auth) {
+    connectionContext.set_protocol(key.getProtocol().getProtocol());
+    std::string euser = key.getAuth().getUser().getPrincipal();
+    std::string ruser = key.getAuth().getUser().getRealUser();
+
+    if (auth.getMethod() != AuthMethod::TOKEN) {
+        UserInformationProto * user = connectionContext.mutable_userinfo();
+        user->set_effectiveuser(euser);
+
+        if (auth.getMethod() != AuthMethod::SIMPLE) {
+            if (!ruser.empty() && ruser != euser) {
+                user->set_realuser(ruser);
+            }
+        }
+    }
+}
+
+void RpcChannelImpl::sendConnectionContent(const RpcAuth & auth) {
+    WriteBuffer buffer;
+    IpcConnectionContextProto connectionContext;
+    RpcRequestHeaderProto rpcHeader;
+    buildConnectionContext(connectionContext, auth);
+    rpcHeader.set_callid(CONNECTION_CONTEXT_CALL_ID);
+    rpcHeader.set_clientid(client.getClientId());
+    rpcHeader.set_retrycount(INVALID_RETRY_COUNT);
+    rpcHeader.set_rpckind(RPC_PROTOCOL_BUFFER);
+    rpcHeader.set_rpcop(RpcRequestHeaderProto_OperationProto_RPC_FINAL_PACKET);
+    RpcContentWrapper wrapper(&rpcHeader, &connectionContext);
+    int size = wrapper.getLength();
+    buffer.writeBigEndian(size);
+    wrapper.writeTo(buffer);
+    sock->writeFully(buffer.getBuffer(0), buffer.getDataSize(0),
+                     key.getConf().getWriteTimeout());
+    lastActivity = lastIdle = steady_clock::now();
+}
+
+RpcRemoteCallPtr RpcChannelImpl::getPendingCall(int32_t id) {
+    unordered_map<int32_t, RpcRemoteCallPtr>::iterator it;
+    it = pendingCalls.find(id);
+
+    if (it == pendingCalls.end()) {
+        THROW(HdfsRpcException,
+              "RPC channel to \"%s:%s\" got protocol mismatch: RPC channel "
+              "cannot find pending call: id = %d.",
+              key.getServer().getHost().c_str(),
+              key.getServer().getPort().c_str(), static_cast<int>(id));
+    }
+
+    RpcRemoteCallPtr rc = it->second;
+    pendingCalls.erase(it);
+    return rc;
+}
+
+bool RpcChannelImpl::getResponse() {
+    int idleTimeout = key.getConf().getMaxIdleTime();
+    int pingTimeout = key.getConf().getPingTimeout();
+    int timeout = key.getConf().getRpcTimeout();
+    int interval = pingTimeout < idleTimeout ? pingTimeout : idleTimeout;
+    interval /= 2;
+    interval = interval < timeout ? interval : timeout;
+    steady_clock::time_point s = steady_clock::now();
+
+    while (client.isRunning()) {
+        if (in->poll(500)) {
+            return true;
+        }
+
+        if (ToMilliSeconds(s, steady_clock::now()) >= interval) {
+            return false;
+        }
+    }
+
+    return false;
+}
+
+static exception_ptr HandlerRpcResponseException(exception_ptr e) {
+    exception_ptr retval = e;
+
+    try {
+        rethrow_exception(e);
+    } catch (const HdfsRpcServerException & e) {
+        UnWrapper < NameNodeStandbyException, RpcNoSuchMethodException,
+                  UnsupportedOperationException, AccessControlException,
+                  SafeModeException, SaslException > unwrapper(e);
+
+        try {
+            unwrapper.unwrap(__FILE__, __LINE__);
+        } catch (const NameNodeStandbyException & e) {
+            retval = current_exception();
+        } catch (const UnsupportedOperationException & e) {
+            retval = current_exception();
+        } catch (const AccessControlException & e) {
+            retval = current_exception();
+        } catch (const SafeModeException & e) {
+            retval = current_exception();
+        } catch (const SaslException & e) {
+            retval = current_exception();
+        } catch (const RpcNoSuchMethodException & e) {
+            retval = current_exception();
+        } catch (const HdfsIOException & e) {
+        }
+    }
+
+    return retval;
+}
+
+void RpcChannelImpl::readOneResponse(bool writeLock) {
+    int readTimeout = key.getConf().getReadTimeout();
+    std::vector<char> buffer(128);
+    RpcResponseHeaderProto curRespHeader;
+    RpcResponseHeaderProto::RpcStatusProto status;
+    uint32_t totalen, headerSize = 0, bodySize = 0;
+    totalen = in->readBigEndianInt32(readTimeout);
+    /*
+     * read response header
+     */
+    headerSize = in->readVarint32(readTimeout);
+    buffer.resize(headerSize);
+    in->readFully(&buffer[0], headerSize, readTimeout);
+
+    if (!curRespHeader.ParseFromArray(&buffer[0], headerSize)) {
+        THROW(HdfsRpcException,
+              "RPC channel to \"%s:%s\" got protocol mismatch: RPC channel "
+              "cannot parse response header.",
+              key.getServer().getHost().c_str(),
+              key.getServer().getPort().c_str())
+    }
+
+    lastActivity = steady_clock::now();
+    status = curRespHeader.status();
+
+    if (RpcResponseHeaderProto_RpcStatusProto_SUCCESS == status) {
+        /*
+         * on success, read response body
+         */
+        RpcRemoteCallPtr rc;
+
+        if (writeLock) {
+            lock_guard<mutex> lock(writeMut);
+            rc = getPendingCall(curRespHeader.callid());
+        } else {
+            rc = getPendingCall(curRespHeader.callid());
+        }
+
+        bodySize = in->readVarint32(readTimeout);
+        buffer.resize(bodySize);
+
+        if (bodySize > 0) {
+            in->readFully(&buffer[0], bodySize, readTimeout);
+        }
+
+        Message *response = rc->getCall().getResponse();
+
+        if (!response->ParseFromArray(&buffer[0], bodySize)) {
+            THROW(HdfsRpcException,
+                  "RPC channel to \"%s:%s\" got protocol mismatch: rpc "
+                  "channel cannot parse response.",
+                  key.getServer().getHost().c_str(),
+                  key.getServer().getPort().c_str())
+        }
+
+        rc->done();
+    } else {
+        /*
+         * on error, read error class and message
+         */
+        std::string errClass, errMessage;
+        errClass = curRespHeader.exceptionclassname();
+        errMessage = curRespHeader.errormsg();
+
+        if (RpcResponseHeaderProto_RpcStatusProto_ERROR == status) {
+            RpcRemoteCallPtr rc;
+            {
+                lock_guard<mutex> lock(writeMut);
+                rc = getPendingCall(curRespHeader.callid());
+            }
+
+            try {
+                THROW(HdfsRpcServerException, "%s: %s",
+                      errClass.c_str(), errMessage.c_str());
+            } catch (HdfsRpcServerException & e) {
+                e.setErrClass(errClass);
+                e.setErrMsg(errMessage);
+                rc->cancel(HandlerRpcResponseException(current_exception()));
+            }
+        } else { /*fatal*/
+            assert(RpcResponseHeaderProto_RpcStatusProto_FATAL == status);
+
+            if (errClass.empty()) {
+                THROW(HdfsRpcException, "%s: %s",
+                      errClass.c_str(), errMessage.c_str());
+            }
+
+            try {
+                THROW(HdfsRpcServerException, "%s: %s", errClass.c_str(),
+                      errMessage.c_str());
+            } catch (HdfsRpcServerException & e) {
+                e.setErrClass(errClass);
+                e.setErrMsg(errMessage);
+                rethrow_exception(
+                      HandlerRpcResponseException(current_exception()));
+            }
+        }
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.h
new file mode 100644
index 0000000..f80019f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.h
@@ -0,0 +1,272 @@
+/**
+ * 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_RPC_RPCCHANNEL_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCHANNEL_H_
+
+#include "Atomic.h"
+#include "DateTime.h"
+#include "ExceptionInternal.h"
+#include "IpcConnectionContext.pb.h"
+#include "RpcCall.h"
+#include "RpcChannelKey.h"
+#include "RpcHeader.pb.h"
+#include "RpcRemoteCall.h"
+#include "SaslClient.h"
+#include "SharedPtr.h"
+#include "Thread.h"
+#include "UnorderedMap.h"
+#include "network/BufferedSocketReader.h"
+#include "network/TcpSocket.h"
+
+#include <google/protobuf/message.h>
+
+namespace hdfs {
+namespace internal {
+
+class RpcClient;
+
+using hadoop::common::RpcSaslProto;
+using hadoop::common::RpcSaslProto_SaslAuth;
+
+class RpcChannel {
+public:
+    /**
+     * Destroy a channel
+     */
+    virtual ~RpcChannel() {
+    }
+
+    /**
+     * The caller finished the rpc call,
+     * this channel may be reused later if immediate is false.
+     * @param immediate Do not reuse the channel any more if immediate is true.
+     */
+    virtual void close(bool immediate) = 0;
+
+    /**
+     * Invoke a rpc call.
+     * @param call The call is to be invoked.
+     * @return The remote call object.
+     */
+    virtual void invoke(const RpcCall &call) = 0;
+
+    /**
+     * Close the channel if it idle expired.
+     * @return true if the channel idle expired.
+     */
+    virtual bool checkIdle() = 0;
+
+    /**
+     * Wait for all reference exiting.
+     * The channel cannot be reused any more.
+     * @pre RpcClient is not running.
+     */
+    virtual void waitForExit() = 0;
+
+    /**
+     * Add reference count to this channel.
+     */
+    virtual void addRef() = 0;
+};
+
+/**
+ * RpcChannel represent a rpc connect to the server.
+ */
+class RpcChannelImpl: public RpcChannel {
+public:
+    /**
+     * Construct a RpcChannelImpl instance.
+     * @param k The key of this channel.
+     */
+    RpcChannelImpl(const RpcChannelKey &k, RpcClient &c);
+
+    /**
+     * Destroy a RpcChannelImpl instance.
+     */
+    ~RpcChannelImpl();
+
+    /**
+     * The caller finished the rpc call,
+     * this channel may be reused later if immediate is false.
+     * @param immediate Do not reuse the channel any more if immediate is true.
+     */
+    void close(bool immediate);
+
+    /**
+     * Invoke a rpc call.
+     * @param call The call is to be invoked.
+     * @return The remote call object.
+     */
+    void invoke(const RpcCall &call);
+
+    /**
+     * Close the channel if it idle expired.
+     * @return true if the channel idle expired.
+     */
+    bool checkIdle();
+
+    /**
+     * Wait for all reference exiting.
+     * The channel cannot be reused any more.
+     * @pre RpcClient is not running.
+     */
+    void waitForExit();
+
+    /**
+     * Add reference count to this channel.
+     */
+    void addRef() {
+        ++refs;
+    }
+
+private:
+    /**
+     * Setup the RPC connection.
+     * @pre Already hold write lock.
+     */
+    void connect();
+
+    /**
+     * Cleanup all pending calls.
+     * @param reason The reason to cancel the call.
+     * @pre Already hold write lock.
+     */
+    void cleanupPendingCalls(exception_ptr reason);
+
+    /**
+     * Send rpc connect protocol header.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     */
+    void sendConnectionHeader();
+
+    /**
+     * Send rpc connection protocol content.
+     */
+    void sendConnectionContent(const RpcAuth &auth);
+
+    /**
+     * Build rpc connect context.
+     */
+    void buildConnectionContext(
+          hadoop::common::IpcConnectionContextProto &connectionContext,
+          const RpcAuth &auth);
+
+    /**
+     * Send ping packet to server.
+     * @throw HdfsNetworkException
+     * @throw HdfsTimeout
+     * @pre Caller should hold the write lock.
+     */
+    void sendPing();
+
+    /**
+     * Send the call message to rpc server.
+     * @param remote The remote call.
+     * @pre Already hold write lock.
+     */
+    void sendRequest(RpcRemoteCallPtr remote);
+
+    /**
+     * Issue a rpc call and check response.
+     * Catch all recoverable error in this function
+     *
+     * @param remote The remote call
+     */
+    exception_ptr invokeInternal(RpcRemoteCallPtr remote);
+
+    /**
+     * Check response, block until get one response.
+     * @pre Channel already hold read lock.
+     */
+    void checkOneResponse();
+
+    /**
+     * read and handle one response.
+     * @pre Channel already hold read lock.
+     */
+    void readOneResponse(bool writeLock);
+
+    /**
+     * Get the call object with given id, and then remove it from pending call list.
+     * @param id The id of the call object to be returned.
+     * @return The call object with given id.
+     * @throw HdfsIOException
+     * @pre Channel already locked.
+     */
+    RpcRemoteCallPtr getPendingCall(int32_t id);
+
+    /**
+     * Check if there is data available for reading on socket.
+     * @return true if response is available.
+     */
+    bool getResponse();
+
+    /**
+     * wake up one caller to check response.
+     * @param id The call id which current caller handled.
+     */
+    void wakeupOneCaller(int32_t id);
+
+    /**
+     * shutdown the RPC connection since error.
+     * @param reason The reason to cancel the call
+     * @pre Already hold write lock.
+     */
+    void shutdown(exception_ptr reason);
+
+    const RpcSaslProto_SaslAuth *createSaslClient(
+        const ::google::protobuf::RepeatedPtrField<RpcSaslProto_SaslAuth> *auths);
+
+    void sendSaslMessage(RpcSaslProto *msg, ::google::protobuf::Message *resp);
+
+    std::string saslEvaluateToken(RpcSaslProto &response, bool serverIsDone);
+
+    RpcAuth setupSaslConnection();
+
+private:
+    /**
+     * Construct a RpcChannelImpl instance for test.
+     * @param key The key of this channel.
+     * @param sock The socket instance.
+     * @param in The BufferedSocketReader instance build on sock.
+     * @param client The RpcClient instance.
+     */
+    RpcChannelImpl(const RpcChannelKey &key, Socket *sock,
+                   BufferedSocketReader *in, RpcClient &client);
+
+private:
+    atomic<int> refs;
+    bool available;
+    mutex readMut;
+    mutex writeMut;
+    RpcChannelKey key;
+    RpcClient &client;
+    shared_ptr<BufferedSocketReader> in;
+    shared_ptr<SaslClient> saslClient;
+    shared_ptr<Socket> sock;
+    steady_clock::time_point lastActivity; // ping is a kind of activity, lastActivity will be updated after ping
+    steady_clock::time_point lastIdle; // ping cannot change idle state. If there is still pending calls, lastIdle is always "NOW".
+    unordered_map<int32_t, RpcRemoteCallPtr> pendingCalls;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCHANNEL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.cc
new file mode 100644
index 0000000..b3600cb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.cc
@@ -0,0 +1,46 @@
+/**
+ * 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 "RpcChannelKey.h"
+
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+RpcChannelKey::RpcChannelKey(const RpcAuth &a, const RpcProtocolInfo &p,
+                             const RpcServerInfo &s, const RpcConfig &c) :
+    auth(a), conf(c), protocol(p), server(s) {
+    const Token *temp = auth.getUser().selectToken(protocol.getTokenKind(),
+                         server.getTokenService());
+
+    if (temp) {
+        token = shared_ptr<Token> (new Token(*temp));
+    }
+}
+
+size_t RpcChannelKey::hash_value() const {
+    size_t tokenHash = token ? token->hash_value() : 0;
+    size_t values[] = { auth.hash_value(), protocol.hash_value(),
+                        server.hash_value(), conf.hash_value(), tokenHash
+                      };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.h
new file mode 100644
index 0000000..72ffcae
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.h
@@ -0,0 +1,86 @@
+/**
+ * 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_RPC_RPCCHANNELKEY_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCHANNELKEY_H_
+
+#include "Hash.h"
+#include "RpcAuth.h"
+#include "RpcConfig.h"
+#include "RpcProtocolInfo.h"
+#include "RpcServerInfo.h"
+#include "SharedPtr.h"
+#include "client/Token.h"
+
+namespace hdfs {
+namespace internal {
+
+class RpcChannelKey {
+public:
+    RpcChannelKey(const RpcAuth &a, const RpcProtocolInfo &p,
+                  const RpcServerInfo &s, const RpcConfig &c);
+
+public:
+    size_t hash_value() const;
+
+    const RpcAuth &getAuth() const {
+        return auth;
+    }
+
+    const RpcConfig &getConf() const {
+        return conf;
+    }
+
+    const RpcProtocolInfo &getProtocol() const {
+        return protocol;
+    }
+
+    const RpcServerInfo &getServer() const {
+        return server;
+    }
+
+    bool operator ==(const RpcChannelKey &other) const {
+        return this->auth == other.auth && this->protocol == other.protocol
+               && this->server == other.server && this->conf == other.conf
+               && ((token == NULL && other.token == NULL)
+                   || (token && other.token && *token == *other.token));
+    }
+
+    const Token &getToken() const {
+        assert(token != NULL);
+        return *token;
+    }
+
+    bool hasToken() {
+        return token != NULL;
+    }
+
+private:
+    const RpcAuth auth;
+    const RpcConfig conf;
+    const RpcProtocolInfo protocol;
+    const RpcServerInfo server;
+    hdfs::internal::shared_ptr<Token> token;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::RpcChannelKey);
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCHANNELKEY_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.cc
new file mode 100644
index 0000000..b0d1caf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.cc
@@ -0,0 +1,188 @@
+/**
+ * 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 "Logger.h"
+#include "SharedPtr.h"
+#include "RpcClient.h"
+#include "Thread.h"
+
+#include <uuid/uuid.h>
+
+namespace hdfs {
+namespace internal {
+
+once_flag RpcClient::once;
+shared_ptr<RpcClient> RpcClient::client;
+
+void RpcClient::createSinglten() {
+    client = shared_ptr < RpcClient > (new RpcClientImpl());
+}
+
+RpcClient & RpcClient::getClient() {
+    call_once(once, &RpcClientImpl::createSinglten);
+    assert(client);
+    return *client;
+}
+
+RpcClientImpl::RpcClientImpl() :
+    cleaning(false), running(true), count(0) {
+    uuid_t id;
+    uuid_generate(id);
+    clientId.resize(sizeof(uuid_t));
+    memcpy(&clientId[0], id, sizeof(uuid_t));
+#ifdef MOCK
+    stub = NULL;
+#endif
+}
+
+RpcClientImpl::~RpcClientImpl() {
+    running = false;
+    cond.notify_all();
+
+    if (cleaner.joinable()) {
+        cleaner.join();
+    }
+
+    close();
+}
+
+void RpcClientImpl::clean() {
+    assert(cleaning);
+
+    try {
+        while (running) {
+            try {
+                unique_lock<mutex> lock(mut);
+                cond.wait_for(lock, seconds(1));
+
+                if (!running || allChannels.empty()) {
+                    break;
+                }
+
+                unordered_map<RpcChannelKey, shared_ptr<RpcChannel> >::iterator s, e;
+                e = allChannels.end();
+
+                for (s = allChannels.begin(); s != e;) {
+                    if (s->second->checkIdle()) {
+                        s->second.reset();
+                        s = allChannels.erase(s);
+                    } else {
+                        ++s;
+                    }
+                }
+            } catch (const HdfsCanceled & e) {
+                /*
+                 * ignore cancel signal here.
+                 */
+            }
+        }
+    } catch (const hdfs::HdfsException & e) {
+        LOG(LOG_ERROR, "RpcClientImpl's idle cleaner exit: %s",
+            GetExceptionDetail(e));
+    } catch (const std::exception & e) {
+        LOG(LOG_ERROR, "RpcClientImpl's idle cleaner exit: %s", e.what());
+    }
+
+    cleaning = false;
+}
+
+void RpcClientImpl::close() {
+    lock_guard<mutex> lock(mut);
+    running = false;
+    unordered_map<RpcChannelKey, shared_ptr<RpcChannel> >::iterator s, e;
+    e = allChannels.end();
+
+    for (s = allChannels.begin(); s != e; ++s) {
+        s->second->waitForExit();
+    }
+
+    allChannels.clear();
+}
+
+bool RpcClientImpl::isRunning() {
+    return running;
+}
+
+RpcChannel & RpcClientImpl::getChannel(const RpcAuth &auth,
+            const RpcProtocolInfo &protocol, const RpcServerInfo &server,
+            const RpcConfig &conf) {
+    shared_ptr<RpcChannel> rc;
+    RpcChannelKey key(auth, protocol, server, conf);
+
+    try {
+        lock_guard<mutex> lock(mut);
+
+        if (!running) {
+            THROW(hdfs::HdfsRpcException,
+                  "Cannot Setup RPC channel to \"%s:%s\" since RpcClient is closing",
+                  key.getServer().getHost().c_str(), key.getServer().getPort().c_str());
+        }
+
+        unordered_map<RpcChannelKey, shared_ptr<RpcChannel> >::iterator it;
+        it = allChannels.find(key);
+
+        if (it != allChannels.end()) {
+            rc = it->second;
+        } else {
+            rc = createChannelInternal(key);
+            allChannels[key] = rc;
+        }
+
+        rc->addRef();
+
+        if (!cleaning) {
+            cleaning = true;
+
+            if (cleaner.joinable()) {
+                cleaner.join();
+            }
+
+            CREATE_THREAD(cleaner, bind(&RpcClientImpl::clean, this));
+        }
+    } catch (const HdfsRpcException & e) {
+        throw;
+    } catch (...) {
+        NESTED_THROW(HdfsRpcException,
+                     "RpcClient failed to create a channel to \"%s:%s\"",
+                     server.getHost().c_str(), server.getPort().c_str());
+    }
+
+    return *rc;
+}
+
+shared_ptr<RpcChannel> RpcClientImpl::createChannelInternal(
+            const RpcChannelKey & key) {
+    shared_ptr<RpcChannel> channel;
+#ifdef MOCK
+
+    if (stub) {
+        channel = shared_ptr < RpcChannel > (stub->getChannel(key, *this));
+    } else {
+        channel = shared_ptr < RpcChannel > (new RpcChannelImpl(key, *this));
+    }
+
+#else
+    channel = shared_ptr<RpcChannel>(new RpcChannelImpl(key, *this));
+#endif
+    return channel;
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.h
new file mode 100644
index 0000000..7a7a65a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.h
@@ -0,0 +1,155 @@
+/**
+ * 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_RPC_RPCCLIENT_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCLIENT_H_
+
+#include "RpcAuth.h"
+#include "RpcCall.h"
+#include "RpcChannel.h"
+#include "RpcChannelKey.h"
+#include "RpcConfig.h"
+#include "RpcProtocolInfo.h"
+#include "RpcServerInfo.h"
+#include "SharedPtr.h"
+#include "Thread.h"
+#include "UnorderedMap.h"
+
+#include <stdint.h>
+#include <string>
+#include <vector>
+
+#ifdef MOCK
+#include "TestRpcChannelStub.h"
+#endif
+
+namespace hdfs {
+namespace internal {
+
+class RpcClient {
+public:
+    /**
+     * Destroy an RpcClient instance.
+     */
+    virtual ~RpcClient() {
+    }
+
+    /**
+     * Get a RPC channel, create a new one if necessary.
+     * @param auth Authentication information used to setup RPC connection.
+     * @param protocol The RPC protocol used in this call.
+     * @param server Remote server information.
+     * @param conf RPC connection configuration.
+     * @param once If true, the RPC channel will not be reused.
+     */
+    virtual RpcChannel &getChannel(const RpcAuth &auth,
+              const RpcProtocolInfo &protocol,
+              const RpcServerInfo &server, const RpcConfig &conf) = 0;
+
+    /**
+     * Check the RpcClient is still running.
+     * @return true if the RpcClient is still running.
+     */
+    virtual bool isRunning() = 0;
+
+    virtual std::string getClientId() const = 0;
+
+    virtual int32_t getCallId() = 0;
+
+public:
+    static RpcClient &getClient();
+    static void createSinglten();
+
+private:
+    static once_flag once;
+    static shared_ptr<RpcClient> client;
+};
+
+class RpcClientImpl: public RpcClient {
+public:
+    /**
+     * Construct a RpcClient.
+     */
+    RpcClientImpl();
+
+    /**
+     * Destroy an RpcClient instance.
+     */
+    ~RpcClientImpl();
+
+    /**
+     * Get a RPC channel, create a new one if necessary.
+     * @param auth Authentication information used to setup RPC connection.
+     * @param protocol The RPC protocol used in this call.
+     * @param server Remote server information.
+     * @param conf RPC connection configuration.
+     * @param once If true, the RPC channel will not be reused.
+     */
+    RpcChannel &getChannel(const RpcAuth &auth,
+          const RpcProtocolInfo &protocol, const RpcServerInfo &server,
+          const RpcConfig &conf);
+
+    /**
+     * Close the RPC channel.
+     */
+    void close();
+
+    /**
+     * Check the RpcClient is still running.
+     * @return true if the RpcClient is still running.
+     */
+    bool isRunning();
+
+    std::string getClientId() const {
+        return clientId;
+    }
+
+    int32_t getCallId() {
+        static mutex mutid;
+        lock_guard<mutex> lock(mutid);
+        ++count;
+        count = count < std::numeric_limits<int32_t>::max() ? count : 0;
+        return count;
+    }
+
+private:
+    shared_ptr<RpcChannel> createChannelInternal(
+        const RpcChannelKey &key);
+
+    void clean();
+
+private:
+    atomic<bool> cleaning;
+    atomic<bool> running;
+    condition_variable cond;
+    int64_t count;
+    mutex mut;
+    std::string clientId;
+    thread cleaner;
+    unordered_map<RpcChannelKey, shared_ptr<RpcChannel> > allChannels;
+
+#ifdef MOCK
+private:
+    hdfs::mock::TestRpcChannelStub *stub;
+#endif
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCLIENT_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.cc
new file mode 100644
index 0000000..84b5648
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.cc
@@ -0,0 +1,36 @@
+/**
+ * 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 "RpcConfig.h"
+
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+size_t RpcConfig::hash_value() const {
+    size_t values[] = { Int32Hasher(maxIdleTime), Int32Hasher(pingTimeout),
+                        Int32Hasher(connectTimeout), Int32Hasher(readTimeout), Int32Hasher(
+                            writeTimeout), Int32Hasher(maxRetryOnConnect), Int32Hasher(
+                            lingerTimeout), Int32Hasher(rpcTimeout), BoolHasher(tcpNoDelay)
+                      };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.h
new file mode 100644
index 0000000..4b32611
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.h
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _HDFS_LIBHDFS3_RPC_RPCCONFIG_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCONFIG_H_
+
+#include "Hash.h"
+#include "SessionConfig.h"
+
+namespace hdfs {
+namespace internal {
+
+class RpcConfig {
+public:
+
+    RpcConfig(const SessionConfig &conf) {
+        connectTimeout = conf.getRpcConnectTimeout();
+        maxIdleTime = conf.getRpcMaxIdleTime();
+        maxRetryOnConnect = conf.getRpcMaxRetryOnConnect();
+        pingTimeout = conf.getRpcPingTimeout();
+        readTimeout = conf.getRpcReadTimeout();
+        writeTimeout = conf.getRpcWriteTimeout();
+        tcpNoDelay = conf.isRpcTcpNoDelay();
+        lingerTimeout = conf.getRpcSocketLingerTimeout();
+        rpcTimeout = conf.getRpcTimeout();
+    }
+
+    size_t hash_value() const;
+
+    int getConnectTimeout() const {
+        return connectTimeout;
+    }
+
+    void setConnectTimeout(int connectTimeout) {
+        this->connectTimeout = connectTimeout;
+    }
+
+    int getMaxIdleTime() const {
+        return maxIdleTime;
+    }
+
+    void setMaxIdleTime(int maxIdleTime) {
+        this->maxIdleTime = maxIdleTime;
+    }
+
+    int getMaxRetryOnConnect() const {
+        return maxRetryOnConnect;
+    }
+
+    void setMaxRetryOnConnect(int maxRetryOnConnect) {
+        this->maxRetryOnConnect = maxRetryOnConnect;
+    }
+
+    int getReadTimeout() const {
+        return readTimeout;
+    }
+
+    void setReadTimeout(int readTimeout) {
+        this->readTimeout = readTimeout;
+    }
+
+    bool isTcpNoDelay() const {
+        return tcpNoDelay;
+    }
+
+    void setTcpNoDelay(bool tcpNoDelay) {
+        this->tcpNoDelay = tcpNoDelay;
+    }
+
+    int getWriteTimeout() const {
+        return writeTimeout;
+    }
+
+    void setWriteTimeout(int writeTimeout) {
+        this->writeTimeout = writeTimeout;
+    }
+
+    int getPingTimeout() const {
+        return pingTimeout;
+    }
+
+    void setPingTimeout(int maxPingTimeout) {
+        this->pingTimeout = maxPingTimeout;
+    }
+
+    int getLingerTimeout() const {
+        return lingerTimeout;
+    }
+
+    void setLingerTimeout(int lingerTimeout) {
+        this->lingerTimeout = lingerTimeout;
+    }
+
+    int getRpcTimeout() const {
+        return rpcTimeout;
+    }
+
+    void setRpcTimeout(int rpcTimeout) {
+        this->rpcTimeout = rpcTimeout;
+    }
+
+    bool operator ==(const RpcConfig &other) const {
+        return this->maxIdleTime == other.maxIdleTime
+               && this->pingTimeout == other.pingTimeout
+               && this->connectTimeout == other.connectTimeout
+               && this->readTimeout == other.readTimeout
+               && this->writeTimeout == other.writeTimeout
+               && this->maxRetryOnConnect == other.maxRetryOnConnect
+               && this->tcpNoDelay == other.tcpNoDelay
+               && this->lingerTimeout == other.lingerTimeout
+               && this->rpcTimeout == other.rpcTimeout;
+    }
+
+private:
+    int maxIdleTime;
+    int pingTimeout;
+    int connectTimeout;
+    int readTimeout;
+    int writeTimeout;
+    int maxRetryOnConnect;
+    int lingerTimeout;
+    int rpcTimeout;
+    bool tcpNoDelay;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::RpcConfig);
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCONFIG_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.cc
new file mode 100644
index 0000000..c9ea2b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.cc
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <google/protobuf/io/coded_stream.h>
+
+#include "RpcContentWrapper.h"
+
+using namespace ::google::protobuf;
+using namespace ::google::protobuf::io;
+
+namespace hdfs {
+namespace internal {
+
+RpcContentWrapper::RpcContentWrapper(Message * header, Message * msg) :
+    header(header), msg(msg) {
+}
+
+int RpcContentWrapper::getLength() {
+    int headerLen, msgLen = 0;
+    headerLen = header->ByteSize();
+    msgLen = msg == NULL ? 0 : msg->ByteSize();
+    return headerLen + CodedOutputStream::VarintSize32(headerLen)
+           + (msg == NULL ?
+              0 : msgLen + CodedOutputStream::VarintSize32(msgLen));
+}
+
+void RpcContentWrapper::writeTo(WriteBuffer & buffer) {
+    int size = header->ByteSize();
+    buffer.writeVarint32(size);
+    header->SerializeToArray(buffer.alloc(size), size);
+
+    if (msg != NULL) {
+        size = msg->ByteSize();
+        buffer.writeVarint32(size);
+        msg->SerializeToArray(buffer.alloc(size), size);
+    }
+}
+
+}
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.h
new file mode 100644
index 0000000..bad8736
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.h
@@ -0,0 +1,45 @@
+/**
+ * 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_RPC_RPCCONTENTWRAPPER_H_
+#define _HDFS_LIBHDFS3_RPC_RPCCONTENTWRAPPER_H_
+
+#include <google/protobuf/message.h>
+
+#include "WriteBuffer.h"
+
+namespace hdfs {
+namespace internal {
+
+class RpcContentWrapper {
+public:
+    RpcContentWrapper(::google::protobuf::Message *header,
+                      ::google::protobuf::Message *msg);
+
+    int getLength();
+    void writeTo(WriteBuffer &buffer);
+
+public:
+    ::google::protobuf::Message *header;
+    ::google::protobuf::Message *msg;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCCONTENTWRAPPER_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.cc
new file mode 100644
index 0000000..89ca9bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.cc
@@ -0,0 +1,30 @@
+/**
+ * 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 "RpcProtocolInfo.h"
+
+namespace hdfs {
+namespace internal {
+
+size_t RpcProtocolInfo::hash_value() const {
+    size_t values[] = { Int32Hasher(version), StringHasher(protocol), StringHasher(tokenKind) };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.h
new file mode 100644
index 0000000..032a58c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.h
@@ -0,0 +1,78 @@
+/**
+ * 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_RPC_RPCPROTOCOLINFO_H_
+#define _HDFS_LIBHDFS3_RPC_RPCPROTOCOLINFO_H_
+
+#include "Hash.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class RpcProtocolInfo {
+public:
+    RpcProtocolInfo(int v, const std::string &p,
+            const std::string &tokenKind) :
+        version(v), protocol(p), tokenKind(tokenKind) {
+    }
+
+    size_t hash_value() const;
+
+    bool operator ==(const RpcProtocolInfo &other) const {
+        return version == other.version && protocol == other.protocol &&
+          tokenKind == other.tokenKind;
+    }
+
+    const std::string &getProtocol() const {
+        return protocol;
+    }
+
+    void setProtocol(const std::string &protocol) {
+        this->protocol = protocol;
+    }
+
+    int getVersion() const {
+        return version;
+    }
+
+    void setVersion(int version) {
+        this->version = version;
+    }
+
+    const std::string getTokenKind() const {
+        return tokenKind;
+    }
+
+    void setTokenKind(const std::string &tokenKind) {
+        this->tokenKind = tokenKind;
+    }
+
+private:
+    int version;
+    std::string protocol;
+    std::string tokenKind;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::RpcProtocolInfo);
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCPROTOCOLINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.cc
new file mode 100644
index 0000000..e53e88a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.cc
@@ -0,0 +1,79 @@
+/**
+ * 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 "ProtobufRpcEngine.pb.h"
+#include "RpcCall.h"
+#include "RpcContentWrapper.h"
+#include "RpcHeader.pb.h"
+#include "RpcRemoteCall.h"
+#include "SharedPtr.h"
+#include "WriteBuffer.h"
+
+#include <google/protobuf/io/coded_stream.h>
+
+#define PING_CALL_ID -4
+
+using namespace google::protobuf::io;
+using namespace ::hadoop::common;
+
+namespace hdfs {
+namespace internal {
+
+void RpcRemoteCall::serialize(const RpcProtocolInfo & protocol,
+                              WriteBuffer & buffer) {
+    RpcRequestHeaderProto rpcHeader;
+    rpcHeader.set_callid(identity);
+    rpcHeader.set_clientid(clientId);
+    rpcHeader.set_retrycount(-1);
+    rpcHeader.set_rpckind(RPC_PROTOCOL_BUFFER);
+    rpcHeader.set_rpcop(RpcRequestHeaderProto_OperationProto_RPC_FINAL_PACKET);
+    RequestHeaderProto requestHeader;
+    requestHeader.set_methodname(call.getName());
+    requestHeader.set_declaringclassprotocolname(protocol.getProtocol());
+    requestHeader.set_clientprotocolversion(protocol.getVersion());
+    RpcContentWrapper wrapper(&requestHeader, call.getRequest());
+    int rpcHeaderLen = rpcHeader.ByteSize();
+    int size = CodedOutputStream::VarintSize32(rpcHeaderLen) + rpcHeaderLen + wrapper.getLength();
+    buffer.writeBigEndian(size);
+    buffer.writeVarint32(rpcHeaderLen);
+    rpcHeader.SerializeToArray(buffer.alloc(rpcHeaderLen), rpcHeaderLen);
+    wrapper.writeTo(buffer);
+}
+
+std::vector<char> RpcRemoteCall::GetPingRequest(const std::string & clientid) {
+    WriteBuffer buffer;
+    std::vector<char> retval;
+    RpcRequestHeaderProto pingHeader;
+    pingHeader.set_callid(PING_CALL_ID);
+    pingHeader.set_clientid(clientid);
+    pingHeader.set_retrycount(INVALID_RETRY_COUNT);
+    pingHeader.set_rpckind(RpcKindProto::RPC_PROTOCOL_BUFFER);
+    pingHeader.set_rpcop(RpcRequestHeaderProto_OperationProto_RPC_FINAL_PACKET);
+    int rpcHeaderLen = pingHeader.ByteSize();
+    int size = CodedOutputStream::VarintSize32(rpcHeaderLen) + rpcHeaderLen;
+    buffer.writeBigEndian(size);
+    buffer.writeVarint32(rpcHeaderLen);
+    pingHeader.SerializeWithCachedSizesToArray(reinterpret_cast<unsigned char *>(buffer.alloc(pingHeader.ByteSize())));
+    retval.resize(buffer.getDataSize(0));
+    memcpy(&retval[0], buffer.getBuffer(0), retval.size());
+    return retval;
+}
+
+}
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.h
new file mode 100644
index 0000000..d5177b5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.h
@@ -0,0 +1,113 @@
+/**
+ * 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_RPC_RPCREMOTECALL_
+#define _HDFS_LIBHDFS3_RPC_RPCREMOTECALL_
+
+#include "DateTime.h"
+#include "ExceptionInternal.h"
+#include "RpcCall.h"
+#include "RpcProtocolInfo.h"
+#include "SharedPtr.h"
+#include "Thread.h"
+#include "WriteBuffer.h"
+
+#include <stdint.h>
+#include <string>
+
+#define INVALID_RETRY_COUNT -1
+
+namespace hdfs {
+namespace internal {
+
+class RpcRemoteCall;
+typedef shared_ptr<RpcRemoteCall> RpcRemoteCallPtr;
+
+class RpcRemoteCall {
+public:
+    RpcRemoteCall(const RpcCall &c, int32_t id, const std::string &clientId) :
+        complete(false), identity(id), call(c), clientId(clientId) {
+    }
+
+    virtual ~RpcRemoteCall() {
+    }
+
+    virtual void cancel(exception_ptr reason) {
+        unique_lock<mutex> lock(mut);
+        complete = true;
+        error = reason;
+        cond.notify_all();
+    }
+
+    virtual void serialize(const RpcProtocolInfo &protocol,
+                           WriteBuffer &buffer);
+
+    const int32_t getIdentity() const {
+        return identity;
+    }
+
+    void wait() {
+        unique_lock<mutex> lock(mut);
+
+        if (!complete) {
+            cond.wait_for(lock, milliseconds(500));
+        }
+    }
+
+    void check() {
+        if (error != exception_ptr()) {
+            rethrow_exception(error);
+        }
+    }
+
+    RpcCall &getCall() {
+        return call;
+    }
+
+    void done() {
+        unique_lock<mutex> lock(mut);
+        complete = true;
+        cond.notify_all();
+    }
+
+    void wakeup() {
+        cond.notify_all();
+    }
+
+    bool finished() {
+        unique_lock<mutex> lock(mut);
+        return complete;
+    }
+
+public:
+    static std::vector<char> GetPingRequest(const std::string &clientid);
+
+private:
+    bool complete;
+    condition_variable cond;
+    const int32_t identity;
+    exception_ptr error;
+    mutex mut;
+    RpcCall call;
+    std::string clientId;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCREMOTECALL_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.cc
new file mode 100644
index 0000000..5a4c1a1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.cc
@@ -0,0 +1,32 @@
+/**
+ * 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 "RpcServerInfo.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+size_t RpcServerInfo::hash_value() const {
+    size_t values[] = { StringHasher(host), StringHasher(port), StringHasher(tokenService) };
+    return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.h
new file mode 100644
index 0000000..fe437b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.h
@@ -0,0 +1,78 @@
+/**
+ * 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_RPC_RPCSERVERINFO_H_
+#define _HDFS_LIBHDFS3_RPC_RPCSERVERINFO_H_
+
+#include "Hash.h"
+
+#include <string>
+#include <sstream>
+
+namespace hdfs {
+namespace internal {
+
+class RpcServerInfo {
+public:
+    RpcServerInfo(const std::string &tokenService, const std::string &h,
+                  const std::string &p) :
+        host(h), port(p), tokenService(tokenService) {
+    }
+
+    RpcServerInfo(const std::string &h, uint32_t p) :
+        host(h) {
+        std::stringstream ss;
+        ss << p;
+        port = ss.str();
+    }
+
+    size_t hash_value() const;
+
+    bool operator ==(const RpcServerInfo &other) const {
+        return this->host == other.host && this->port == other.port &&
+            tokenService == other.tokenService;
+    }
+
+    const std::string &getTokenService() const {
+        return tokenService;
+    }
+
+    const std::string &getHost() const {
+        return host;
+    }
+
+    const std::string &getPort() const {
+        return port;
+    }
+
+    void setTokenService(const std::string &tokenService) {
+        this->tokenService = tokenService;
+    }
+
+private:
+    std::string host;
+    std::string port;
+    std::string tokenService;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::RpcServerInfo);
+
+#endif /* _HDFS_LIBHDFS3_RPC_RPCSERVERINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.cc
new file mode 100644
index 0000000..bfe6868
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.cc
@@ -0,0 +1,157 @@
+/**
+ * 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 <algorithm>
+#include <cctype>
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "SaslClient.h"
+
+#define SASL_SUCCESS 0
+
+namespace hdfs {
+namespace internal {
+
+SaslClient::SaslClient(const RpcSaslProto_SaslAuth & auth, const Token & token,
+                       const std::string & principal) :
+    complete(false) {
+    int rc;
+    ctx = NULL;
+    RpcAuth method = RpcAuth(RpcAuth::ParseMethod(auth.method()));
+    rc = gsasl_init(&ctx);
+
+    if (rc != GSASL_OK) {
+        THROW(HdfsIOException, "cannot initialize libgsasl");
+    }
+
+    switch (method.getMethod()) {
+    case AuthMethod::KERBEROS:
+        initKerberos(auth, principal);
+        break;
+
+    case AuthMethod::TOKEN:
+        initDigestMd5(auth, token);
+        break;
+
+    default:
+        THROW(HdfsIOException, "unknown auth method.");
+        break;
+    }
+}
+
+SaslClient::~SaslClient() {
+    if (session != NULL) {
+        gsasl_finish(session);
+    }
+
+    if (ctx != NULL) {
+        gsasl_done(ctx);
+    }
+}
+
+void SaslClient::initKerberos(const RpcSaslProto_SaslAuth & auth,
+                              const std::string & principal) {
+    int rc;
+
+    /* Create new authentication session. */
+    if ((rc = gsasl_client_start(ctx, auth.mechanism().c_str(), &session)) != GSASL_OK) {
+        THROW(HdfsIOException, "Cannot initialize client (%d): %s", rc,
+              gsasl_strerror(rc));
+    }
+
+    gsasl_property_set(session, GSASL_SERVICE, auth.protocol().c_str());
+    gsasl_property_set(session, GSASL_AUTHID, principal.c_str());
+    gsasl_property_set(session, GSASL_HOSTNAME, auth.serverid().c_str());
+}
+
+std::string Base64Encode(const std::string & in) {
+    char * temp;
+    size_t len;
+    std::string retval;
+    int rc = gsasl_base64_to(in.c_str(), in.size(), &temp, &len);
+
+    if (rc != GSASL_OK) {
+        throw std::bad_alloc();
+    }
+
+    if (temp) {
+        retval = temp;
+        free(temp);
+    }
+
+    if (!temp || retval.length() != len) {
+        THROW(HdfsIOException, "SaslClient: Failed to encode string to base64");
+    }
+
+    return retval;
+}
+
+void SaslClient::initDigestMd5(const RpcSaslProto_SaslAuth & auth,
+                               const Token & token) {
+    int rc;
+
+    if ((rc = gsasl_client_start(ctx, auth.mechanism().c_str(), &session)) != GSASL_OK) {
+        THROW(HdfsIOException, "Cannot initialize client (%d): %s", rc, gsasl_strerror(rc));
+    }
+
+    std::string password = Base64Encode(token.getPassword());
+    std::string identifier = Base64Encode(token.getIdentifier());
+    gsasl_property_set(session, GSASL_PASSWORD, password.c_str());
+    gsasl_property_set(session, GSASL_AUTHID, identifier.c_str());
+    gsasl_property_set(session, GSASL_HOSTNAME, auth.serverid().c_str());
+    gsasl_property_set(session, GSASL_SERVICE, auth.protocol().c_str());
+}
+
+std::string SaslClient::evaluateChallenge(const std::string & challenge) {
+    int rc;
+    char * output = NULL;
+    size_t outputSize;
+    std::string retval;
+    rc = gsasl_step(session, &challenge[0], challenge.size(), &output,
+                    &outputSize);
+
+    if (rc == GSASL_NEEDS_MORE || rc == GSASL_OK) {
+        retval.resize(outputSize);
+        memcpy(&retval[0], output, outputSize);
+
+        if (output) {
+            free(output);
+        }
+    } else {
+        if (output) {
+            free(output);
+        }
+
+        THROW(AccessControlException, "Failed to evaluate challenge: %s", gsasl_strerror(rc));
+    }
+
+    if (rc == GSASL_OK) {
+        complete = true;
+    }
+
+    return retval;
+}
+
+bool SaslClient::isComplete() {
+    return complete;
+}
+
+}
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.h
new file mode 100644
index 0000000..5e95ebd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.h
@@ -0,0 +1,62 @@
+/**
+ * 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_RPC_SASLCLIENT_H_
+#define _HDFS_LIBHDFS3_RPC_SASLCLIENT_H_
+
+#include "RpcAuth.h"
+#include "RpcHeader.pb.h"
+#include "client/Token.h"
+#include "network/Socket.h"
+
+#include <gsasl.h>
+#include <string>
+
+#define SWITCH_TO_SIMPLE_AUTH -88
+
+namespace hdfs {
+namespace internal {
+
+using hadoop::common::RpcSaslProto_SaslAuth;
+
+class SaslClient {
+public:
+    SaslClient(const RpcSaslProto_SaslAuth &auth, const Token &token,
+               const std::string &principal);
+
+    ~SaslClient();
+
+    std::string evaluateChallenge(const std::string &challenge);
+
+    bool isComplete();
+
+private:
+    void initKerberos(const RpcSaslProto_SaslAuth &auth,
+                      const std::string &principal);
+    void initDigestMd5(const RpcSaslProto_SaslAuth &auth, const Token &token);
+
+private:
+    Gsasl *ctx;
+    Gsasl_session *session;
+    bool complete;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_RPC_SASLCLIENT_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/BlockLocalPathInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/BlockLocalPathInfo.h
new file mode 100644
index 0000000..332bb00
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/BlockLocalPathInfo.h
@@ -0,0 +1,64 @@
+/**
+ * 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_SERVER_BLOCKLOCALPATHINFO_H_
+#define _HDFS_LIBHDFS3_SERVER_BLOCKLOCALPATHINFO_H_
+
+#include "ExtendedBlock.h"
+
+#include <string>
+
+namespace hdfs {
+namespace internal {
+
+class BlockLocalPathInfo {
+public:
+    const ExtendedBlock &getBlock() const {
+        return block;
+    }
+
+    void setBlock(const ExtendedBlock &block) {
+        this->block = block;
+    }
+
+    const char *getLocalBlockPath() const {
+        return localBlockPath.c_str();
+    }
+
+    void setLocalBlockPath(const char *localBlockPath) {
+        this->localBlockPath = localBlockPath;
+    }
+
+    const char *getLocalMetaPath() const {
+        return localMetaPath.c_str();
+    }
+
+    void setLocalMetaPath(const char *localMetaPath) {
+        this->localMetaPath = localMetaPath;
+    }
+
+private:
+    ExtendedBlock block;
+    std::string localBlockPath;
+    std::string localMetaPath;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_BLOCKLOCALPATHINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.cc
new file mode 100644
index 0000000..c997b5d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.cc
@@ -0,0 +1,99 @@
+/**
+ * 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 "ClientDatanodeProtocol.pb.h"
+#include "Datanode.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "RpcHelper.h"
+
+#include <string>
+
+#define DATANODE_VERSION 1
+#define DATANODE_PROTOCOL "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol"
+#define BLOCK_TOKEN_KIND "HDFS_BLOCK_TOKEN"
+
+using namespace google::protobuf;
+using namespace hadoop::common;
+using namespace hadoop::hdfs;
+
+namespace hdfs {
+namespace internal {
+
+DatanodeImpl::DatanodeImpl(const std::string &host, uint32_t port,
+                           const SessionConfig &c, const RpcAuth &a) :
+    auth(a), client(RpcClient::getClient()), conf(c), protocol(
+        DATANODE_VERSION, DATANODE_PROTOCOL, BLOCK_TOKEN_KIND),
+        server(host, port) {
+    server.setTokenService("");
+}
+
+void DatanodeImpl::invoke(const RpcCall & call, bool reuse) {
+    RpcChannel & channel = client.getChannel(auth, protocol, server, conf);
+
+    try {
+        channel.invoke(call);
+    } catch (const HdfsFailoverException & e) {
+        //Datanode do not have HA configuration.
+        channel.close(true);
+        rethrow_if_nested(e);
+        assert(false && "HdfsFailoverException should be always a "
+               "wrapper of other exception");
+    } catch (...) {
+        channel.close(true);
+        throw;
+    }
+
+    channel.close(!reuse);
+}
+
+int64_t DatanodeImpl::getReplicaVisibleLength(const ExtendedBlock &b) {
+    try {
+        GetReplicaVisibleLengthRequestProto request;
+        GetReplicaVisibleLengthResponseProto response;
+        Build(b, request.mutable_block());
+        invoke(RpcCall(true, "getReplicaVisibleLength",
+                       &request, &response), false);
+        return response.length();
+    } catch (const HdfsRpcServerException & e) {
+        UnWrapper<ReplicaNotFoundException, HdfsIOException> unwraper(e);
+        unwraper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+void DatanodeImpl::getBlockLocalPathInfo(const ExtendedBlock &block,
+        const Token &token, BlockLocalPathInfo &info) {
+    try {
+        ExtendedBlock eb;
+        GetBlockLocalPathInfoRequestProto request;
+        GetBlockLocalPathInfoResponseProto response;
+        Build(block, request.mutable_block());
+        Build(token, request.mutable_token());
+        invoke(RpcCall(true, "getBlockLocalPathInfo", &request, &response), true);
+        Convert(eb, response.block());
+        info.setBlock(eb);
+        info.setLocalBlockPath(response.localpath().c_str());
+        info.setLocalMetaPath(response.localmetapath().c_str());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<ReplicaNotFoundException, HdfsIOException> unwraper(e);
+        unwraper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.h
new file mode 100644
index 0000000..1ecafa2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.h
@@ -0,0 +1,103 @@
+/**
+ * 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_SERVER_DATANODE_H_
+#define _HDFS_LIBHDFS3_SERVER_DATANODE_H_
+
+#include "BlockLocalPathInfo.h"
+#include "client/Token.h"
+#include "ExtendedBlock.h"
+#include "rpc/RpcAuth.h"
+#include "rpc/RpcCall.h"
+#include "rpc/RpcClient.h"
+#include "rpc/RpcConfig.h"
+#include "rpc/RpcProtocolInfo.h"
+#include "rpc/RpcServerInfo.h"
+#include "SessionConfig.h"
+
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+class Datanode {
+public:
+    virtual ~Datanode() {
+    }
+
+    /**
+     * Return the visible length of a replica.
+     * @param b The block which visible length is to be returned.
+     * @return the visible length of the block.
+     * @throw ReplicaNotFoundException
+     * @throw HdfsIOException
+     */
+    //Idempotent
+    virtual int64_t getReplicaVisibleLength(const ExtendedBlock &b)
+    /*throw (ReplicaNotFoundException, HdfsIOException)*/ = 0;
+
+    /**
+     * Retrieves the path names of the block file and metadata file stored on the
+     * local file system.
+     *
+     * In order for this method to work, one of the following should be satisfied:
+     * <ul>
+     * <li>
+     * The client user must be configured at the datanode to be able to use this
+     * method.</li>
+     * <li>
+     * When security is enabled, kerberos authentication must be used to connect
+     * to the datanode.</li>
+     * </ul>
+     *
+     * @param block The specified block on the local datanode
+     * @param token The block access token.
+     * @param info Output the BlockLocalPathInfo of block.
+     * @throw HdfsIOException
+     */
+    //Idempotent
+    virtual void getBlockLocalPathInfo(const ExtendedBlock &block,
+                                       const Token & token, BlockLocalPathInfo &info)
+    /*throw (HdfsIOException)*/ = 0;
+};
+
+class DatanodeImpl: public Datanode {
+public:
+    DatanodeImpl(const std::string & host, uint32_t port, const SessionConfig & c,
+                 const RpcAuth & a);
+
+    virtual int64_t getReplicaVisibleLength(const ExtendedBlock &b);
+
+    virtual void getBlockLocalPathInfo(const ExtendedBlock &block,
+                                       const Token & token, BlockLocalPathInfo &info);
+
+private:
+    void invoke(const RpcCall & call, bool reuse);
+
+private:
+    RpcAuth auth;
+    RpcClient &client;
+    RpcConfig conf;
+    RpcProtocolInfo protocol;
+    RpcServerInfo server;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_DATANODE_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/DatanodeInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/DatanodeInfo.h
new file mode 100644
index 0000000..21ce4a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/DatanodeInfo.h
@@ -0,0 +1,126 @@
+/**
+ * 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_SERVER_DATANODEINFO_H_
+#define _HDFS_LIBHDFS3_SERVER_DATANODEINFO_H_
+
+#include <stdint.h>
+#include <string>
+#include <sstream>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * This class extends the primary identifier of a Datanode with ephemeral
+ * state, eg usage information, current administrative state, and the
+ * network location that is communicated to clients.
+ */
+class DatanodeInfo {
+public:
+    const std::string &getHostName() const {
+        return hostName;
+    }
+
+    void setHostName(const std::string &hostName) {
+        this->hostName = hostName;
+    }
+
+    uint32_t getInfoPort() const {
+        return infoPort;
+    }
+
+    void setInfoPort(uint32_t infoPort) {
+        this->infoPort = infoPort;
+    }
+
+    const std::string &getIpAddr() const {
+        return ipAddr;
+    }
+
+    void setIpAddr(const std::string &ipAddr) {
+        this->ipAddr = ipAddr;
+    }
+
+    uint32_t getIpcPort() const {
+        return ipcPort;
+    }
+
+    void setIpcPort(uint32_t ipcPort) {
+        this->ipcPort = ipcPort;
+    }
+
+    const std::string &getDatanodeId() const {
+        return datanodeId;
+    }
+
+    void setDatanodeId(const std::string &storageId) {
+        this->datanodeId = storageId;
+    }
+
+    uint32_t getXferPort() const {
+        return xferPort;
+    }
+
+    void setXferPort(uint32_t xferPort) {
+        this->xferPort = xferPort;
+    }
+
+    const std::string formatAddress() const {
+        std::stringstream ss;
+        ss << hostName << "(" << getIpAddr() << ")";
+        return ss.str();
+    }
+
+    bool operator <(const DatanodeInfo &other) const {
+        return datanodeId < other.datanodeId;
+    }
+
+    bool operator ==(const DatanodeInfo &other) const {
+        return this->datanodeId == other.datanodeId
+               && this->ipAddr == other.ipAddr;
+    }
+
+    const std::string &getLocation() const {
+        return location;
+    }
+
+    void setLocation(const std::string &location) {
+        this->location = location;
+    }
+
+    std::string getXferAddr() const {
+        std::stringstream ss;
+        ss << getIpAddr() << ":" << getXferPort();
+        return ss.str();
+    }
+
+private:
+    uint32_t xferPort;
+    uint32_t infoPort;
+    uint32_t ipcPort;
+    std::string ipAddr;
+    std::string hostName;
+    std::string datanodeId;
+    std::string location;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_DATANODEINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/ExtendedBlock.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/ExtendedBlock.h
new file mode 100644
index 0000000..e070658
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/ExtendedBlock.h
@@ -0,0 +1,95 @@
+/**
+ * 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_SERVER_EXTENDEDBLOCK_H_
+#define _HDFS_LIBHDFS3_SERVER_EXTENDEDBLOCK_H_
+
+#include "Hash.h"
+
+#include <string>
+#include <sstream>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * Identifies a Block uniquely across the block pools
+ */
+class ExtendedBlock {
+public:
+    ExtendedBlock() :
+        blockId(0), generationStamp(0), numBytes(0) {
+    }
+
+    int64_t getBlockId() const {
+        return blockId;
+    }
+
+    void setBlockId(int64_t blockId) {
+        this->blockId = blockId;
+    }
+
+    int64_t getGenerationStamp() const {
+        return generationStamp;
+    }
+
+    void setGenerationStamp(int64_t generationStamp) {
+        this->generationStamp = generationStamp;
+    }
+
+    int64_t getNumBytes() const {
+        return numBytes;
+    }
+
+    void setNumBytes(int64_t numBytes) {
+        this->numBytes = numBytes;
+    }
+
+    const std::string &getPoolId() const {
+        return poolId;
+    }
+
+    void setPoolId(const std::string &poolId) {
+        this->poolId = poolId;
+    }
+
+    const std::string toString() const {
+        std::stringstream ss;
+        ss << "[block pool ID: " << poolId << " block ID " << blockId << "_"
+           << generationStamp << "]";
+        return ss.str();
+    }
+
+    size_t hash_value() const {
+        size_t values[] = { Int64Hasher(blockId), StringHasher(poolId) };
+        return CombineHasher(values, sizeof(values) / sizeof(values[0]));
+    }
+
+private:
+    int64_t blockId;
+    int64_t generationStamp;
+    int64_t numBytes;
+    std::string poolId;
+};
+
+}
+}
+
+HDFS_HASH_DEFINE(::hdfs::internal::ExtendedBlock);
+
+#endif /* _HDFS_LIBHDFS3_SERVER_EXTENDEDBLOCK_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlock.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlock.h
new file mode 100644
index 0000000..76ee762
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlock.h
@@ -0,0 +1,105 @@
+/**
+ * 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_SERVER_LOCATEDBLOCK_H_
+#define _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCK_H_
+
+#include "client/Token.h"
+#include "DatanodeInfo.h"
+#include "ExtendedBlock.h"
+
+#include <vector>
+#include <stdint.h>
+
+namespace hdfs {
+namespace internal {
+
+/**
+ * Associates a block with the Datanodes that contain its replicas
+ * and other block metadata (E.g. the file offset associated with this
+ * block, whether it is corrupt, security token, etc).
+ */
+class LocatedBlock: public ExtendedBlock {
+public:
+    LocatedBlock() :
+        offset(0), corrupt(false) {
+    }
+
+    LocatedBlock(int64_t position) :
+        offset(position), corrupt(false) {
+    }
+
+    bool isCorrupt() const {
+        return corrupt;
+    }
+
+    void setCorrupt(bool corrupt) {
+        this->corrupt = corrupt;
+    }
+
+    const std::vector<DatanodeInfo> &getLocations() const {
+        return locs;
+    }
+
+    std::vector<DatanodeInfo> &mutableLocations() {
+        return locs;
+    }
+
+    void setLocations(const std::vector<DatanodeInfo> &locs) {
+        this->locs = locs;
+    }
+
+    int64_t getOffset() const {
+        return offset;
+    }
+
+    void setOffset(int64_t offset) {
+        this->offset = offset;
+    }
+
+    const Token &getToken() const {
+        return token;
+    }
+
+    void setToken(const Token &token) {
+        this->token = token;
+    }
+
+    bool operator <(const LocatedBlock &that) const {
+        return this->offset < that.offset;
+    }
+
+    const std::vector<std::string> &getStorageIDs() const {
+        return storageIDs;
+    }
+
+    std::vector<std::string> &mutableStorageIDs() {
+        return storageIDs;
+    }
+private:
+    int64_t offset;
+    bool corrupt;
+    std::vector<DatanodeInfo> locs;
+    std::vector<std::string> storageIDs;
+    Token token;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCK_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc
new file mode 100644
index 0000000..794a0ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc
@@ -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.
+ */
+
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "LocatedBlock.h"
+#include "LocatedBlocks.h"
+
+#include <algorithm>
+#include <cassert>
+#include <iostream>
+
+namespace hdfs {
+namespace internal {
+
+const LocatedBlock *LocatedBlocksImpl::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.
+         */
+        bound = std::lower_bound(blocks.begin(), blocks.end(), target,
+                                 std::less<LocatedBlock>());
+        assert(bound == blocks.end() || bound->getOffset() >= position);
+        LocatedBlock * retval = NULL;
+
+        if (bound == blocks.end()) {
+            retval = &blocks.back();
+        } else if (bound->getOffset() > position) {
+            --bound;
+            retval = &(*bound);
+        } else {
+            retval = &(*bound);
+        }
+
+        if (position < retval->getOffset()
+                || position >= retval->getOffset() + retval->getNumBytes()) {
+            return NULL;
+        }
+
+        return retval;
+    } else {
+        return lastBlock.get();
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h
new file mode 100644
index 0000000..8d7281b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h
@@ -0,0 +1,111 @@
+/**
+ * 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_SERVER_LOCATEDBLOCKS_H_
+#define _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCKS_H_
+
+#include "LocatedBlock.h"
+#include "common/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;
+    }
+
+    void setFileLength(int64_t fileLength) {
+        this->fileLength = fileLength;
+    }
+
+    bool isLastBlockComplete() const {
+        return lastBlockComplete;
+    }
+
+    void setIsLastBlockComplete(bool lastBlockComplete) {
+        this->lastBlockComplete = lastBlockComplete;
+    }
+
+    shared_ptr<LocatedBlock> getLastBlock() {
+        assert(!lastBlockComplete);
+        return lastBlock;
+    }
+
+    void setLastBlock(shared_ptr<LocatedBlock> lastBlock) {
+        this->lastBlock = lastBlock;
+    }
+
+    bool isUnderConstruction() const {
+        return underConstruction;
+    }
+
+    void setUnderConstruction(bool underConstruction) {
+        this->underConstruction = underConstruction;
+    }
+
+    const LocatedBlock * findBlock(int64_t position);
+
+    std::vector<LocatedBlock> & getBlocks() {
+        return blocks;
+    }
+
+private:
+    bool lastBlockComplete;
+    bool underConstruction;
+    int64_t fileLength;
+    shared_ptr<LocatedBlock> lastBlock;
+    std::vector<LocatedBlock> blocks;
+
+};
+
+}
+}
+#endif /* _HDFS_LIBHDFS3_SERVER_LOCATEDBLOCKS_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h
new file mode 100644
index 0000000..4dbf517
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h
@@ -0,0 +1,770 @@
+/**
+ * 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_SERVER_NAMENODE_H_
+#define _HDFS_LIBHDFS3_SERVER_NAMENODE_H_
+
+#include "client/FileStatus.h"
+#include "client/Permission.h"
+#include "DatanodeInfo.h"
+#include "Exception.h"
+#include "ExtendedBlock.h"
+#include "LocatedBlock.h"
+#include "LocatedBlocks.h"
+#include "rpc/RpcAuth.h"
+#include "rpc/RpcCall.h"
+#include "rpc/RpcClient.h"
+#include "rpc/RpcConfig.h"
+#include "rpc/RpcProtocolInfo.h"
+#include "rpc/RpcServerInfo.h"
+#include "SessionConfig.h"
+
+#include <stdint.h>
+#include <vector>
+
+namespace hdfs {
+namespace internal {
+
+class Namenode {
+public:
+    /**
+     * Destroy the namenode.
+     */
+    virtual ~Namenode() {
+    }
+
+    /**
+     * 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>
+     * Return {//link LocatedBlocks} which contains
+     * file length, blocks and their locations.
+     * DataNode locations for each block are sorted by
+     * the distance to the client's address.
+     * <p>
+     * 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 file length and array of blocks with their locations
+     * @param lbs output the returned blocks
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> does not exist
+     * @throw UnresolvedLinkException If <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual void getBlockLocations(const std::string &src, int64_t offset,
+                                   int64_t length, LocatedBlocks &lbs)
+      /* throw (AccessControlException,
+             FileNotFoundException, UnresolvedLinkException,
+             HdfsIOException) */ = 0;
+
+    /**
+     * Create a new file entry in the namespace.
+     * <p>
+     * This will create an empty file specified by the source path.
+     * 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
+     * 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[])}
+     *
+     * @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.
+     * @param createParent create missing parent directory if true
+     * @param replication block replication factor.
+     * @param blockSize maximum block size.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw AlreadyBeingCreatedException if the path does not exist.
+     * @throw DSQuotaExceededException If file creation violates disk space
+     *           quota restriction
+     * @throw FileAlreadyExistsException If file <code>src</code> already exists
+     * @throw FileNotFoundException If parent of <code>src</code> does not exist
+     *           and <code>createParent</code> is false
+     * @throw ParentNotDirectoryException If parent of <code>src</code> is not a
+     *           directory.
+     * @throw NSQuotaExceededException If file creation violates name space
+     *           quota restriction
+     * @throw SafeModeException create not allowed in safemode
+     * @throw UnresolvedLinkException If <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     *
+     * RuntimeExceptions:
+     * @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,
+             AlreadyBeingCreatedException, DSQuotaExceededException,
+             FileAlreadyExistsException, FileNotFoundException,
+             NSQuotaExceededException, ParentNotDirectoryException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Append to the end of the file.
+     * @param src path of the file being created.
+     * @param clientName name of the current client.
+     * @param information about the last partial block if any.
+     * @param lb output the returned block.s
+     *
+     * @throw AccessControlException if permission to append file is
+     * denied by the system. As usually on the client side the exception will
+     * be wrapped into {//link org.apache.hadoop.ipc.RemoteException}.
+     * Allows appending to an existing file if the server is
+     * configured with the parameter dfs.support.append set to true, otherwise
+     * throw an HdfsIOException.
+     *
+     * @throw AccessControlException If permission to append to file is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw DSQuotaExceededException If append violates disk space quota
+     *           restriction
+     * @throw SafeModeException append not allowed in safemode
+     * @throw UnresolvedLinkException If <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred.
+     *
+     * RuntimeExceptions:
+     * @throw UnsupportedOperationException if append is not supported
+     */
+    virtual shared_ptr<LocatedBlock> append(const std::string &src,
+                                            const std::string &clientName)
+    /* throw (AccessControlException,
+             DSQuotaExceededException, FileNotFoundException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Set replication for an existing file.
+     * <p>
+     * The NameNode sets replication to the new value and returns.
+     * The actual block replication is not expected to be performed during
+     * this method call. The blocks will be populated or removed in the
+     * background as the result of the routine block maintenance procedures.
+     *
+     * @param src file name
+     * @param replication new replication
+     *
+     * @param true if successful) = 0;
+     *         false if file does not exist or is a directory
+     *
+     * @throw AccessControlException If access is denied
+     * @throw DSQuotaExceededException If replication violates disk space
+     *           quota restriction
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException not allowed in safemode
+     * @throw UnresolvedLinkException if <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual bool setReplication(const std::string &src, short replication)
+    /* throw (AccessControlException, DSQuotaExceededException,
+     FileNotFoundException, SafeModeException, UnresolvedLinkException,
+     HdfsIOException) */ = 0;
+
+    /**
+     * Set permissions for an existing file/directory.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException not allowed in safemode
+     * @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,
+             FileNotFoundException, SafeModeException,
+             UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Set Owner of a path (i.e. a file or a directory).
+     * The parameters username and groupname cannot both be null.
+     * @param src
+     * @param username If it is null, the original username remains unchanged.
+     * @param groupname If it is null, the original groupname remains unchanged.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException not allowed in safemode
+     * @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,
+             FileNotFoundException, SafeModeException,
+             UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * 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.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException file <code>src</code> is not found
+     * @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,
+             FileNotFoundException, UnresolvedLinkException,
+             HdfsIOException) */ = 0;
+
+    /**
+     * 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 clientName the name of the client that adds the block
+     * @param previous  previous block
+     * @param excludeNodes a list of nodes that should not be
+     * allocated for the current block
+     *
+     * @param LocatedBlock allocated block information.
+     * @param lb output the returned block.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw NotReplicatedYetException previous blocks of the file are not
+     *           replicated yet. Blocks cannot be added until replication
+     *           completes.
+     * @throw SafeModeException create not allowed in safemode
+     * @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,
+     NotReplicatedYetException, SafeModeException,
+     UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * 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
+     * @param clientName the name of the client
+     *
+     * @param the located block.
+     * @param output the returned block.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException create not allowed in safemode
+     * @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,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * 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.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException create not allowed in safemode
+     * @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,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * The client wants to report corrupted blocks (blocks with specified
+     * locations on datanodes).
+     * @param blocks Array of located blocks to report
+     */
+    //Idempotent
+    /*    virtual void reportBadBlocks(const std::vector<LocatedBlock> &blocks)
+         throw (HdfsIOException)  = 0;*/
+
+    /**
+     * Rename an item in the file system namespace.
+     * @param src existing file or directory name.
+     * @param dst new name.
+     * @param true if successful, or false if the old name does not exist
+     * or if the new name already belongs to the namespace.
+     *
+     * @throw HdfsIOException an I/O error occurred
+     */
+    virtual bool rename(const std::string &src, const std::string &dst)
+    /* throw (UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Moves blocks from srcs to trg and delete srcs
+     *
+     * @param trg existing file
+     * @param srcs - list of existing files (same block size, same replication)
+     * @throw HdfsIOException if some arguments are invalid
+     * @throw UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
+     *           contains a symlink
+     */
+    /*    virtual void concat(const std::string &trg,
+                            const std::vector<std::string> &srcs)  throw (HdfsIOException,
+                 UnresolvedLinkException)  = 0;*/
+
+    /**
+     * Rename src to dst.
+     * <ul>
+     * <li>Fails if src is a file and dst is a directory.
+     * <li>Fails if src is a directory and dst is a file.
+     * <li>Fails if the parent of dst does not exist or is a file.
+     * </ul>
+     * <p>
+     * Without OVERWRITE option, rename fails if the dst already exists.
+     * With OVERWRITE option, rename overwrites the dst, if it is a file
+     * or an empty directory. Rename fails if dst is a non-empty directory.
+     * <p>
+     * This implementation of rename is atomic.
+     * <p>
+     * @param src existing file or directory name.
+     * @param dst new name.
+     * @param options Rename options
+     *
+     * @throw AccessControlException If access is denied
+     * @throw DSQuotaExceededException If rename violates disk space
+     *           quota restriction
+     * @throw FileAlreadyExistsException If <code>dst</code> already exists and
+     *           <code>options</options> has {//link Rename#OVERWRITE} option
+     *           false.
+     * @throw FileNotFoundException If <code>src</code> does not exist
+     * @throw NSQuotaExceededException If rename violates namespace
+     *           quota restriction
+     * @throw ParentNotDirectoryException If parent of <code>dst</code>
+     *           is not a directory
+     * @throw SafeModeException rename not allowed in safemode
+     * @throw UnresolvedLinkException If <code>src</code> or
+     *           <code>dst</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    /*virtual void rename2(const std::string &src, const std::string &dst)
+     throw (AccessControlException, DSQuotaExceededException,
+     FileAlreadyExistsException, FileNotFoundException,
+     NSQuotaExceededException, ParentNotDirectoryException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) = 0;*/
+
+    /**
+     * Delete the given file or directory from the file system.
+     * <p>
+     * same as delete but provides a way to avoid accidentally
+     * deleting non empty directories programmatically.
+     * @param src existing name
+     * @param recursive if true deletes a non empty directory recursively,
+     * else throw( an exception.
+     * @param true only if the existing file or directory was actually removed
+     * from the file system.
+     *
+     * @throw AccessControlException If access is denied
+     * @throw FileNotFoundException If file <code>src</code> is not found
+     * @throw SafeModeException create not allowed in safemode
+     * @throw UnresolvedLinkException If <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    virtual bool deleteFile(const std::string &src, bool recursive)
+    /* throw (AccessControlException, FileNotFoundException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Create a directory (or hierarchy of directories) with the given
+     * name and permission.
+     *
+     * @param src The path of the directory being created
+     * @param masked The masked permission of the directory being created
+     * @param createParent create missing parent directory if true
+     *
+     * @param True if the operation success.
+     *
+     * @throw AccessControlException If access is denied
+     * @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 ParentNotDirectoryException If parent of <code>src</code>
+     *           is not a directory
+     * @throw SafeModeException create not allowed in safemode
+     * @throw UnresolvedLinkException If <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred.
+     *
+     * RunTimeExceptions:
+     * @throw InvalidPathException If <code>src</code> is invalid
+     */
+    //Idempotent
+    virtual bool mkdirs(const std::string &src, const Permission &masked,
+                        bool createParent) /* throw (AccessControlException,
+             FileAlreadyExistsException, FileNotFoundException,
+             NSQuotaExceededException, ParentNotDirectoryException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * 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 a partial listing starting after startAfter
+     * @param dl append the returned directories.
+     *
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException file <code>src</code> is not found
+     * @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,
+             FileNotFoundException, UnresolvedLinkException,
+             HdfsIOException) */ = 0;
+
+    /**
+     * Client programs can cause stateful changes in the NameNode
+     * that affect other clients.  A client may obtain a file and
+     * neither abandon nor complete it.  A client might hold a series
+     * of locks that prevent other clients from proceeding.
+     * Clearly, it would be bad if a client held a bunch of locks
+     * that it never gave up.  This can happen easily if the client
+     * dies unexpectedly.
+     * <p>
+     * So, the NameNode will revoke the locks and live file-creates
+     * for clients that it thinks have died.  A client tells the
+     * NameNode that it is still alive by periodically calling
+     * renewLease().  If a certain amount of time passes since
+     * the last call to renewLease(), the NameNode assumes the
+     * client has died.
+     *
+     * @throw AccessControlException permission denied
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual void renewLease(const std::string &clientName)
+    /* throw (AccessControlException, HdfsIOException) */ = 0;
+
+    /**
+     * Start lease recovery.
+     * Lightweight NameNode operation to trigger lease recovery
+     *
+     * @param src path of the file to start lease recovery
+     * @param clientName name of the current client
+     * @param true if the file is already closed
+     * @throw HdfsIOException
+     */
+    //Idempotent
+    virtual bool recoverLease(const std::string &src,
+                              const std::string &clientName) = 0;
+
+    /**
+     * 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> [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>
+     * <li> [4] contains number of blocks with a corrupt replica. </li>
+     * <li> [5] contains number of blocks without any good replicas left. </li>
+     * </ul>
+     * Use  constants like {//link #GET_STATS_CAPACITY_IDX} in place of
+     * actual numbers to index into the array.
+     */
+    //Idempotent
+    virtual std::vector<int64_t> getFsStats() /* throw (HdfsIOException) */ = 0;
+
+    /**
+     * Dumps namenode data structures into specified file. If the file
+     * already exists, then append.
+     *
+     * @throw HdfsIOException
+     */
+    /*    virtual void metaSave(
+            const std::string &filename)  throw (HdfsIOException)  = 0;*/
+
+    /**
+     * Get the file info for a specific file or directory.
+     * @param src The const std::string &representation of the path to the file
+     *
+     * @param object containing information regarding the file
+     *         or null if file not found
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException file <code>src</code> is not found
+     * @throw UnresolvedLinkException if the path contains a symlink.
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual FileStatus getFileInfo(const std::string &src)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Get the file info for a specific file or directory. If the path
+     * refers to a symlink then the FileStatus of the symlink is returned.
+     * @param src The const std::string &representation of the path to the file
+     *
+     * @param object containing information regarding the file
+     *         or null if file not found
+     *
+     * @throw AccessControlException permission denied
+     * @throw UnresolvedLinkException if <code>src</code> contains a symlink
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    /*    virtual FileStatus getFileLinkInfo(const std::string &src)
+         throw (AccessControlException, UnresolvedLinkException,
+         HdfsIOException)  = 0;*/
+
+    /**
+     * Get {//link ContentSummary} rooted at the specified directory.
+     * @param path The const std::string &representation of the path
+     *
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException file <code>path</code> is not found
+     * @throw UnresolvedLinkException if <code>path</code> contains a symlink.
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //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 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
+     *                       this directory.
+     * <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 will not be changed, and (3) {//link HdfsConstants#QUOTA_RESET}
+     * implies the quota will be reset. Any other value is a runtime error.
+     *
+     * @throw AccessControlException permission denied
+     * @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 HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    /*
+        virtual void setQuota(const std::string &path, int64_t namespaceQuota,
+                              int64_t diskspaceQuota)  throw (AccessControlException,
+                 FileNotFoundException, UnresolvedLinkException,
+                 HdfsIOException)  = 0;
+    */
+
+    /**
+     * 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
+     * @param client The const std::string &representation of the client
+     *
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException file <code>src</code> is not found
+     * @throw UnresolvedLinkException if <code>src</code> contains a symlink.
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual void fsync(const std::string &src, const std::string &client)
+    /* 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
+     *              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
+     *              by this call.
+     *
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException file <code>src</code> is not found
+     * @throw UnresolvedLinkException if <code>src</code> contains a symlink.
+     * @throw HdfsIOException If an I/O error occurred
+     */
+    //Idempotent
+    virtual void setTimes(const std::string &src, int64_t mtime, int64_t atime)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */ = 0;
+
+    /**
+     * Create symlink to a file or directory.
+     * @param target The path of the destination that the
+     *               link points to.
+     * @param link The path of the link being created.
+     * @param dirPerm permissions to use when creating parent directories
+     * @param createParent - if true then missing parent dirs are created
+     *                       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
+     *           and <code>createParent</code> is false
+     * @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,
+                 FileAlreadyExistsException, FileNotFoundException,
+                 ParentNotDirectoryException, SafeModeException,
+                 UnresolvedLinkException, HdfsIOException)  = 0;*/
+
+    /**
+     * Return the target of the given symlink. If there is an intermediate
+     * symlink in the path (ie a symlink leading up to the final path component)
+     * then the given path is returned with this symlink resolved.
+     *
+     * @param path The path with a link that needs resolution.
+     * @param The path after resolving the first symbolic link in the path.
+     * @throw AccessControlException permission denied
+     * @throw FileNotFoundException If <code>path</code> does not exist
+     * @throw HdfsIOException If the given path does not refer to a symlink
+     *           or an I/O error occurred
+     */
+    //Idempotent
+    /*    virtual std::string getLinkTarget(const std::string &path)
+         throw (AccessControlException, FileNotFoundException,
+         HdfsIOException)  = 0;*/
+
+    /**
+     * 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
+     * @param clientName the name of the client
+     * @param a located block with a new generation stamp and an access token
+     * @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;
+
+    /**
+     * Update a pipeline for a block under construction
+     *
+     * @param clientName the name of the client
+     * @param oldBlock the old block
+     * @param newBlock the new block containing new generation stamp and length
+     * @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;
+
+    /**
+      * Get a valid Delegation Token.
+      *
+      * @param renewer the designated renewer for the token
+      * @return Token<DelegationTokenIdentifier>
+      * @throws IOException
+      */
+    virtual Token getDelegationToken(const std::string &renewer)
+    /* throws IOException*/ = 0;
+
+    /**
+     * Renew an existing delegation token.
+     *
+     * @param token delegation token obtained earlier
+     * @return the new expiration time
+     * @throws IOException
+     */
+    virtual int64_t renewDelegationToken(const Token &token)
+    /*throws IOException*/ = 0;
+
+    /**
+     * Cancel an existing delegation token.
+     *
+     * @param token delegation token
+     * @throws IOException
+     */
+    virtual void cancelDelegationToken(const Token &token)
+    /*throws IOException*/ = 0;
+
+    /**
+     * close the namenode connection.
+     */
+    virtual void close() {};
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_NAMENODE_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.cc
new file mode 100644
index 0000000..3d499e7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.cc
@@ -0,0 +1,730 @@
+/**
+ * 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 "ClientNamenodeProtocol.pb.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "Namenode.h"
+#include "NamenodeImpl.h"
+#include "rpc/RpcCall.h"
+#include "rpc/RpcClient.h"
+#include "RpcHelper.h"
+
+#define NAMENODE_VERSION 1
+#define NAMENODE_PROTOCOL "org.apache.hadoop.hdfs.protocol.ClientProtocol"
+#define DELEGATION_TOKEN_KIND "HDFS_DELEGATION_TOKEN"
+
+using namespace google::protobuf;
+using namespace hadoop::common;
+using namespace hadoop::hdfs;
+
+namespace hdfs {
+namespace internal {
+
+NamenodeImpl::NamenodeImpl(const char *host, const char *port,
+        const std::string &tokenService, const SessionConfig &c,
+        const RpcAuth &a) :
+    auth(a), client(RpcClient::getClient()), conf(c), protocol(
+        NAMENODE_VERSION, NAMENODE_PROTOCOL, DELEGATION_TOKEN_KIND),
+        server(tokenService, host, port) {
+}
+
+NamenodeImpl::~NamenodeImpl() {
+}
+
+void NamenodeImpl::invoke(const RpcCall &call) {
+    RpcChannel &channel = client.getChannel(auth, protocol, server, conf);
+
+    try {
+        channel.invoke(call);
+    } catch (...) {
+        channel.close(false);
+        throw;
+    }
+
+    channel.close(false);
+}
+
+//Idempotent
+void NamenodeImpl::getBlockLocations(const std::string &src, int64_t offset,
+        int64_t length, LocatedBlocks &lbs) /* throw (AccessControlException,
+         FileNotFoundException, UnresolvedLinkException, HdfsIOException) */ {
+    try {
+        GetBlockLocationsRequestProto request;
+        GetBlockLocationsResponseProto response;
+        request.set_length(length);
+        request.set_offset(offset);
+        request.set_src(src);
+        invoke(RpcCall(true, "getBlockLocations", &request, &response));
+        Convert(lbs, response.locations());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+void NamenodeImpl::create(const std::string &src, const Permission &masked,
+                          const std::string &clientName, int flag, bool createParent,
+                          short replication, int64_t blockSize)
+    /* throw (AccessControlException,
+         AlreadyBeingCreatedException, DSQuotaExceededException,
+         FileAlreadyExistsException, FileNotFoundException,
+         NSQuotaExceededException, ParentNotDirectoryException,
+          UnresolvedLinkException, HdfsIOException) */{
+    try {
+        CreateRequestProto request;
+        CreateResponseProto response;
+        request.set_blocksize(blockSize);
+        request.set_clientname(clientName);
+        request.set_createflag(flag);
+        request.set_createparent(createParent);
+        request.set_replication(replication);
+        request.set_src(src);
+        Build(masked, request.mutable_masked());
+        invoke(RpcCall(false, "create", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < AlreadyBeingCreatedException,
+                  DSQuotaExceededException, FileAlreadyExistsException,
+                  FileNotFoundException, NSQuotaExceededException,
+                  ParentNotDirectoryException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+shared_ptr<LocatedBlock> NamenodeImpl::append(const std::string &src,
+        const std::string &clientName)
+/* throw (AlreadyBeingCreatedException, DSQuotaExceededException,
+ FileNotFoundException,
+ UnresolvedLinkException, HdfsIOException) */{
+    try {
+        AppendRequestProto request;
+        AppendResponseProto response;
+        request.set_clientname(clientName);
+        request.set_src(src);
+        invoke(RpcCall(false, "append", &request, &response));
+
+        if (response.has_block()) {
+            return Convert(response.block());
+        } else {
+            return shared_ptr<LocatedBlock>();
+        }
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < AlreadyBeingCreatedException, AccessControlException,
+                  DSQuotaExceededException, FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+bool NamenodeImpl::setReplication(const std::string &src, short replication)
+/* throw (DSQuotaExceededException,
+ FileNotFoundException,  UnresolvedLinkException,
+ HdfsIOException) */{
+    try {
+        SetReplicationRequestProto request;
+        SetReplicationResponseProto response;
+        request.set_src(src.c_str());
+        request.set_replication(static_cast<uint32>(replication));
+        invoke(RpcCall(true, "setReplication", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < DSQuotaExceededException,
+                  FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+void NamenodeImpl::setPermission(const std::string &src,
+                                 const Permission &permission) /* throw (AccessControlException,
+         FileNotFoundException,
+         UnresolvedLinkException, HdfsIOException) */{
+    try {
+        SetPermissionRequestProto request;
+        SetPermissionResponseProto response;
+        request.set_src(src);
+        Build(permission, request.mutable_permission());
+        invoke(RpcCall(true, "setPermission", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+void NamenodeImpl::setOwner(const std::string &src,
+                            const std::string &username, const std::string &groupname)
+/* throw (FileNotFoundException,
+  UnresolvedLinkException, HdfsIOException) */{
+    try {
+        SetOwnerRequestProto request;
+        SetOwnerResponseProto response;
+        request.set_src(src);
+
+        if (!username.empty()) {
+            request.set_username(username);
+        }
+
+        if (!groupname.empty()) {
+            request.set_groupname(groupname);
+        }
+
+        invoke(RpcCall(true, "setOwner", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+void NamenodeImpl::abandonBlock(const ExtendedBlock &b,
+                                const std::string &src, const std::string &holder)
+/* throw (FileNotFoundException,
+ UnresolvedLinkException, HdfsIOException) */{
+    try {
+        AbandonBlockRequestProto request;
+        AbandonBlockResponseProto response;
+        request.set_holder(holder);
+        request.set_src(src);
+        Build(b, request.mutable_b());
+        invoke(RpcCall(false, "abandonBlock", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+shared_ptr<LocatedBlock> NamenodeImpl::addBlock(const std::string &src,
+        const std::string &clientName, const ExtendedBlock *previous,
+        const std::vector<DatanodeInfo> &excludeNodes)
+/* throw (FileNotFoundException,
+ NotReplicatedYetException,
+ UnresolvedLinkException, HdfsIOException) */{
+    try {
+        AddBlockRequestProto request;
+        AddBlockResponseProto response;
+        request.set_clientname(clientName);
+        request.set_src(src);
+
+        if (previous) {
+            Build(*previous, request.mutable_previous());
+        }
+
+        if (excludeNodes.size()) {
+            Build(excludeNodes, request.mutable_excludenodes());
+        }
+
+        invoke(RpcCall(true, "addBlock", &request, &response));
+        return Convert(response.block());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  NotReplicatedYetException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+shared_ptr<LocatedBlock> NamenodeImpl::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 ( FileNotFoundException,
+  UnresolvedLinkException, HdfsIOException) */{
+    try {
+        GetAdditionalDatanodeRequestProto request;
+        GetAdditionalDatanodeResponseProto response;
+        request.set_src(src);
+        Build(existings, request.mutable_existings());
+        Build(storageIDs, request.mutable_existingstorageuuids());
+        Build(excludes, request.mutable_excludes());
+        Build(blk, request.mutable_blk());
+        request.set_clientname(clientName);
+        request.set_numadditionalnodes(numAdditionalNodes);
+        invoke(RpcCall(true, "getAdditionalDatanode", &request, &response));
+        return Convert(response.block());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper <
+        FileNotFoundException,
+        NotReplicatedYetException,
+        UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+bool NamenodeImpl::complete(const std::string &src,
+                            const std::string &clientName, const ExtendedBlock *last)
+/* throw (FileNotFoundException,
+  UnresolvedLinkException, HdfsIOException) */{
+    try {
+        CompleteRequestProto request;
+        CompleteResponseProto response;
+        request.set_clientname(clientName);
+        request.set_src(src);
+
+        if (last) {
+            Build(*last, request.mutable_last());
+        }
+
+        invoke(RpcCall(false, "complete", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+/*void NamenodeImpl::reportBadBlocks(const std::vector<LocatedBlock> &blocks)
+ throw (HdfsIOException) {
+    try {
+        ReportBadBlocksRequestProto request;
+        ReportBadBlocksResponseProto response;
+        Build(blocks, request.mutable_blocks());
+        invoke(RpcCall(true, "reportBadBlocks", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+bool NamenodeImpl::rename(const std::string &src, const std::string &dst)
+      /* throw (UnresolvedLinkException, HdfsIOException) */{
+    try {
+        RenameRequestProto request;
+        RenameResponseProto response;
+        request.set_src(src);
+        request.set_dst(dst);
+        invoke(RpcCall(false, "rename", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<UnresolvedLinkException, HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+/*void NamenodeImpl::concat(const std::string &trg,
+          const std::vector<std::string> &srcs)
+      throw (UnresolvedLinkException, HdfsIOException) {
+    try {
+        ConcatRequestProto request;
+        ConcatResponseProto response;
+        request.set_trg(trg);
+        Build(srcs, request.mutable_srcs());
+        invoke(RpcCall(false, "concat", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<UnresolvedLinkException, HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+bool NamenodeImpl::deleteFile(const std::string &src, bool recursive)
+      /* throw (FileNotFoundException, UnresolvedLinkException,
+       * HdfsIOException) */ {
+    try {
+        DeleteRequestProto request;
+        DeleteResponseProto response;
+        request.set_src(src);
+        request.set_recursive(recursive);
+        invoke(RpcCall(false, "delete", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+bool NamenodeImpl::mkdirs(const std::string &src, const Permission &masked,
+                          bool createParent) /* throw (AccessControlException,
+         FileAlreadyExistsException, FileNotFoundException,
+         NSQuotaExceededException, ParentNotDirectoryException,
+          UnresolvedLinkException, HdfsIOException) */{
+    try {
+        MkdirsRequestProto request;
+        MkdirsResponseProto response;
+        request.set_src(src);
+        request.set_createparent(createParent);
+        Build(masked, request.mutable_masked());
+        invoke(RpcCall(true, "mkdirs", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileAlreadyExistsException,
+                  FileNotFoundException, NSQuotaExceededException,
+                  ParentNotDirectoryException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+bool NamenodeImpl::getListing(const std::string &src,
+          const std::string &startAfter, bool needLocation,
+          std::vector<FileStatus> &dl) /* throw (AccessControlException,
+         FileNotFoundException, UnresolvedLinkException, HdfsIOException) */ {
+    try {
+        GetListingRequestProto request;
+        GetListingResponseProto response;
+        request.set_src(src);
+        request.set_startafter(startAfter);
+        request.set_needlocation(needLocation);
+        invoke(RpcCall(true, "getListing", &request, &response));
+
+        if (response.has_dirlist()) {
+            const DirectoryListingProto &lists = response.dirlist();
+            Convert(dl, lists);
+            return lists.remainingentries() > 0;
+        }
+
+        THROW(FileNotFoundException, "%s not found.", src.c_str());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+void NamenodeImpl::renewLease(const std::string &clientName)
+          /* throw (HdfsIOException) */{
+    try {
+        RenewLeaseRequestProto request;
+        RenewLeaseResponseProto response;
+        request.set_clientname(clientName);
+        invoke(RpcCall(true, "renewLease", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+bool NamenodeImpl::recoverLease(const std::string &src,
+           const std::string &clientName) /* throw (HdfsIOException) */ {
+    try {
+        RecoverLeaseRequestProto request;
+        RecoverLeaseResponseProto response;
+        request.set_src(src);
+        request.set_clientname(clientName);
+        invoke(RpcCall(true, "recoverLease", &request, &response));
+        return response.result();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+
+    return false;
+}
+
+//Idempotent
+std::vector<int64_t> NamenodeImpl::getFsStats() {
+      /* throw (HdfsIOException) */
+    try {
+        GetFsStatusRequestProto request;
+        GetFsStatsResponseProto response;
+        invoke(RpcCall(true, "getFsStats", &request, &response));
+        std::vector<int64_t> retval;
+        retval.push_back(response.capacity());
+        retval.push_back(response.used());
+        retval.push_back(response.remaining());
+        retval.push_back(response.under_replicated());
+        retval.push_back(response.corrupt_blocks());
+        retval.push_back(response.missing_blocks());
+        return retval;
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(
+                      e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+
+    return std::vector<int64_t>();
+}
+
+/*void NamenodeImpl::metaSave(const std::string &filename)
+ throw (HdfsIOException) {
+    try {
+        MetaSaveRequestProto request;
+        MetaSaveResponseProto response;
+        request.set_filename(filename);
+        invoke(RpcCall(true, "metaSave", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+FileStatus NamenodeImpl::getFileInfo(const std::string &src)
+/* throw (FileNotFoundException,
+ UnresolvedLinkException, HdfsIOException) */{
+    FileStatus retval;
+
+    try {
+        GetFileInfoRequestProto request;
+        GetFileInfoResponseProto response;
+        request.set_src(src);
+        invoke(RpcCall(true, "getFileInfo", &request, &response));
+
+        if (response.has_fs()) {
+            Convert(retval, response.fs());
+            assert(src.find_last_of('/') != src.npos);
+            const char *path = src.c_str() + src.find_last_of('/') + 1;
+            path = src == "/" ? "/" : path;
+            retval.setPath(path);
+            return retval;
+        }
+
+        THROW(FileNotFoundException, "Path %s does not exist.", src.c_str());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+/*FileStatus NamenodeImpl::getFileLinkInfo(const std::string &src)
+ throw (UnresolvedLinkException, HdfsIOException) {
+    FileStatus fileStatus;
+
+    try {
+        GetFileLinkInfoRequestProto request;
+        GetFileLinkInfoResponseProto response;
+        request.set_src(src);
+        invoke(RpcCall(true, "getFileLinkInfo", &request, &response));
+
+        if (response.has_fs()) {
+            Convert(fileStatus, response.fs());
+            return fileStatus;
+        }
+
+        THROW(FileNotFoundException, "Path %s does not exist.", src.c_str());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < UnresolvedLinkException,
+                  HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+/*ContentSummary NamenodeImpl::getContentSummary(const std::string &path)
+ throw (FileNotFoundException,
+ UnresolvedLinkException, HdfsIOException) {
+    ContentSummary contentSummary;
+
+    try {
+        GetContentSummaryRequestProto request;
+        GetContentSummaryResponseProto response;
+        request.set_path(path);
+        invoke(RpcCall(true, "getContentSummary", &request, &response));
+
+        if (response.has_summary()) {
+            Convert(contentSummary, response.summary());
+            return contentSummary;
+        }
+
+        THROW(FileNotFoundException, "Path %s does not exist.", path.c_str());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+/*void NamenodeImpl::setQuota(const std::string &path, int64_t namespaceQuota,
+                            int64_t diskspaceQuota)  throw (AccessControlException,
+         FileNotFoundException, UnresolvedLinkException, HdfsIOException) {
+    try {
+        SetQuotaRequestProto request;
+        SetQuotaResponseProto response;
+        request.set_path(path);
+        request.set_namespacequota(namespaceQuota);
+        request.set_diskspacequota(diskspaceQuota);
+        invoke(RpcCall(true, "diskspaceQuota", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+void NamenodeImpl::fsync(const std::string &src, const std::string &client)
+/* throw (FileNotFoundException,
+ UnresolvedLinkException, HdfsIOException) */{
+    try {
+        FsyncRequestProto request;
+        FsyncResponseProto response;
+        request.set_client(client);
+        request.set_src(src);
+        invoke(RpcCall(true, "fsync", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+//Idempotent
+void NamenodeImpl::setTimes(const std::string &src, int64_t mtime,
+                            int64_t atime) /* throw (FileNotFoundException,
+         UnresolvedLinkException, HdfsIOException) */{
+    try {
+        SetTimesRequestProto request;
+        SetTimesResponseProto response;
+        request.set_src(src);
+        request.set_mtime(mtime);
+        request.set_atime(atime);
+        invoke(RpcCall(true, "setTimes", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper < FileNotFoundException,
+                  UnresolvedLinkException, HdfsIOException > unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+/*void NamenodeImpl::createSymlink(const std::string &target,
+              const std::string &link, const Permission &dirPerm,
+              bool createParent)  throw (AccessControlException,
+         FileAlreadyExistsException, FileNotFoundException,
+         ParentNotDirectoryException,
+         UnresolvedLinkException, HdfsIOException) {
+    try {
+        CreateSymlinkRequestProto request;
+        CreateSymlinkResponseProto response;
+        request.set_target(target);
+        request.set_link(link);
+        request.set_createparent(createParent);
+        Build(dirPerm, request.mutable_dirperm());
+        invoke(RpcCall(true, "createSymlink", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<FileNotFoundException, HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+/*std::string NamenodeImpl::getLinkTarget(const std::string &path)
+ throw (FileNotFoundException, HdfsIOException) {
+    try {
+        GetLinkTargetRequestProto request;
+        GetLinkTargetResponseProto response;
+        request.set_path(path);
+        invoke(RpcCall(true, "getLinkTarget", &request, &response));
+        return response.targetpath();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<FileNotFoundException, HdfsIOException> unwrapper(
+            e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}*/
+
+//Idempotent
+shared_ptr<LocatedBlock> NamenodeImpl::updateBlockForPipeline(
+        const ExtendedBlock &block, const std::string &clientName)
+/* throw (HdfsIOException) */{
+    try {
+        UpdateBlockForPipelineRequestProto request;
+        UpdateBlockForPipelineResponseProto response;
+        request.set_clientname(clientName);
+        Build(block, request.mutable_block());
+        invoke(RpcCall(true, "updateBlockForPipeline", &request, &response));
+        return Convert(response.block());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+void NamenodeImpl::updatePipeline(const std::string &clientName,
+          const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
+          const std::vector<DatanodeInfo> &newNodes,
+          const std::vector<std::string> &storageIDs) {
+    /* throw (HdfsIOException) */
+    try {
+        UpdatePipelineRequestProto request;
+        UpdatePipelineResponseProto response;
+        request.set_clientname(clientName);
+        Build(oldBlock, request.mutable_oldblock());
+        Build(newBlock, request.mutable_newblock());
+        Build(newNodes, request.mutable_newnodes());
+        Build(storageIDs, request.mutable_storageids());
+        invoke(RpcCall(false, "updatePipeline", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+Token NamenodeImpl::getDelegationToken(const std::string &renewer) {
+    try {
+        GetDelegationTokenRequestProto request;
+        GetDelegationTokenResponseProto response;
+        request.set_renewer(renewer);
+        invoke(RpcCall(true, "getDelegationToken", &request, &response));
+        return Convert(response.token());
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+int64_t NamenodeImpl::renewDelegationToken(const Token &token) {
+    try {
+        RenewDelegationTokenRequestProto request;
+        RenewDelegationTokenResponseProto response;
+        Build(token, request.mutable_token());
+        invoke(RpcCall(true, "renewDelegationToken", &request, &response));
+        return response.newexpirytime();
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsInvalidBlockToken, HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+void NamenodeImpl::cancelDelegationToken(const Token &token) {
+    try {
+        CancelDelegationTokenRequestProto request;
+        CancelDelegationTokenResponseProto response;
+        Build(token, request.mutable_token());
+        invoke(RpcCall(true, "cancelDelegationToken", &request, &response));
+    } catch (const HdfsRpcServerException &e) {
+        UnWrapper<HdfsInvalidBlockToken, HdfsIOException> unwrapper(e);
+        unwrapper.unwrap(__FILE__, __LINE__);
+    }
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h
new file mode 100644
index 0000000..809d643
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h
@@ -0,0 +1,222 @@
+/**
+ * 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_SERVER_NAMENODEIMPL_H_
+#define _HDFS_LIBHDFS3_SERVER_NAMENODEIMPL_H_
+
+#include "Namenode.h"
+
+namespace hdfs {
+namespace internal {
+
+class NamenodeImpl: public Namenode {
+public:
+    NamenodeImpl(const char *host, const char *port,
+            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,
+             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,
+             AlreadyBeingCreatedException, DSQuotaExceededException,
+             FileAlreadyExistsException, FileNotFoundException,
+             NSQuotaExceededException, ParentNotDirectoryException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    shared_ptr<LocatedBlock> append(const std::string &src, const std::string &clientName)
+    /* throw (AccessControlException,
+             DSQuotaExceededException, FileNotFoundException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    bool setReplication(const std::string &src, short replication)
+    /* throw (AccessControlException, DSQuotaExceededException,
+     FileNotFoundException, SafeModeException, UnresolvedLinkException,
+     HdfsIOException) */;
+
+    //Idempotent
+    void setPermission(const std::string &src, const Permission &permission)
+    /* throw (AccessControlException, FileNotFoundException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    //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,
+             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,
+     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,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    bool complete(const std::string &src, const std::string &clientName,
+                  const ExtendedBlock *last) /* throw (AccessControlException,
+             FileNotFoundException, SafeModeException,
+             UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    void reportBadBlocks(const std::vector<LocatedBlock> &blocks)
+    /* throw (HdfsIOException) */;
+
+    bool rename(const std::string &src, const std::string &dst)
+    /* throw (UnresolvedLinkException, HdfsIOException) */;
+
+    void concat(const std::string &trg, const std::vector<std::string> &srcs)
+    /* throw (HdfsIOException, UnresolvedLinkException) */;
+
+    /*void rename2(const std::string &src, const std::string &dst)
+     throw (AccessControlException, DSQuotaExceededException,
+     FileAlreadyExistsException, FileNotFoundException,
+     NSQuotaExceededException, ParentNotDirectoryException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) ;*/
+
+    bool deleteFile(const std::string &src, bool recursive)
+    /* throw (AccessControlException, FileNotFoundException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    bool mkdirs(const std::string &src, const Permission &masked,
+                bool createParent) /* throw (AccessControlException,
+             FileAlreadyExistsException, FileNotFoundException,
+             NSQuotaExceededException, ParentNotDirectoryException,
+             SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    bool getListing(const std::string &src, const std::string &startAfter,
+                    bool needLocation, std::vector<FileStatus> &dl)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    void renewLease(const std::string &clientName)
+    /* throw (AccessControlException, HdfsIOException) */;
+
+    //Idempotent
+    bool recoverLease(const std::string &src, const std::string &clientName)
+    /* throw (HdfsIOException) */;
+
+    //Idempotent
+    std::vector<int64_t> getFsStats() /* throw (HdfsIOException) */;
+
+    void metaSave(const std::string &filename) /* throw (HdfsIOException) */;
+
+    //Idempotent
+    FileStatus getFileInfo(const std::string &src)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    FileStatus getFileLinkInfo(const std::string &src)
+    /* throw (AccessControlException, UnresolvedLinkException,
+     HdfsIOException) */;
+
+    /*    //Idempotent
+        ContentSummary getContentSummary(const std::string &path)
+         throw (AccessControlException, FileNotFoundException,
+         UnresolvedLinkException, HdfsIOException) ;*/
+
+    //Idempotent
+    void setQuota(const std::string &path, int64_t namespaceQuota,
+                  int64_t diskspaceQuota) /* throw (AccessControlException,
+             FileNotFoundException, UnresolvedLinkException,
+             HdfsIOException) */;
+
+    //Idempotent
+    void fsync(const std::string &src, const std::string &client)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    void setTimes(const std::string &src, int64_t mtime, int64_t atime)
+    /* throw (AccessControlException, FileNotFoundException,
+     UnresolvedLinkException, HdfsIOException) */;
+
+    void createSymlink(const std::string &target, const std::string &link,
+                       const Permission &dirPerm, bool createParent)
+    /* throw (AccessControlException, FileAlreadyExistsException,
+     FileNotFoundException, ParentNotDirectoryException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) */;
+
+    //Idempotent
+    std::string getLinkTarget(const std::string &path)
+    /* throw (AccessControlException, FileNotFoundException,
+     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 std::vector<DatanodeInfo> &newNodes,
+                        const std::vector<std::string> &storageIDs) /* throw (HdfsIOException) */;
+
+    //Idempotent
+    Token getDelegationToken(const std::string &renewer)
+    /* throws IOException*/;
+
+    //Idempotent
+    int64_t renewDelegationToken(const Token &token)
+    /*throws IOException*/;
+
+    //Idempotent
+    void cancelDelegationToken(const Token &token)
+    /*throws IOException*/;
+
+private:
+    void invoke(const RpcCall &call);
+
+private:
+    RpcAuth auth;
+    RpcClient &client;
+    RpcConfig conf;
+    RpcProtocolInfo protocol;
+    RpcServerInfo server;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_NAMENODEIMPL_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc
new file mode 100644
index 0000000..1a0a655
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc
@@ -0,0 +1,59 @@
+/**
+ * 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 "NamenodeInfo.h"
+#include "StringUtil.h"
+#include "XmlConfig.h"
+
+#include <string>
+#include <vector>
+
+using namespace hdfs::internal;
+
+namespace hdfs {
+
+NamenodeInfo::NamenodeInfo() {
+}
+
+const char *const DFS_NAMESERVICES = "dfs.nameservices";
+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, "")));
+    }
+
+    return retval;
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h
new file mode 100644
index 0000000..f317161
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h
@@ -0,0 +1,59 @@
+/**
+ * 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_LIBHDFS_SERVER_NAMENODEINFO_H_
+#define _HDFS_LIBHDFS_SERVER_NAMENODEINFO_H_
+
+#include "XmlConfig.h"
+
+#include <string>
+#include <vector>
+
+namespace hdfs {
+
+class NamenodeInfo {
+public:
+    NamenodeInfo();
+
+    const std::string &getHttpAddr() const {
+        return http_addr;
+    }
+
+    void setHttpAddr(const std::string &httpAddr) {
+        http_addr = httpAddr;
+    }
+
+    const std::string &getRpcAddr() const {
+        return rpc_addr;
+    }
+
+    void setRpcAddr(const std::string &rpcAddr) {
+        rpc_addr = rpcAddr;
+    }
+
+    static std::vector<NamenodeInfo> GetHANamenodeInfo(
+          const std::string &service, const Config &conf);
+
+private:
+    std::string rpc_addr;
+    std::string http_addr;
+};
+
+}
+
+#endif /* _HDFS_LIBHDFS_SERVER_NAMENODEINFO_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.cc
new file mode 100644
index 0000000..020f719
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.cc
@@ -0,0 +1,491 @@
+/**
+ * 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 "Logger.h"
+#include "NamenodeImpl.h"
+#include "NamenodeProxy.h"
+#include "StringUtil.h"
+
+#include <string>
+
+#include <sys/fcntl.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <sys/file.h>
+
+namespace hdfs {
+namespace internal {
+
+static uint32_t GetInitNamenodeIndex(const std::string &id) {
+    std::string path = "/tmp/";
+    path += id;
+    int fd;
+    uint32_t index = 0;
+    /*
+     * try create the file
+     */
+    fd = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, 0666);
+
+    if (fd < 0) {
+        if (errno == EEXIST) {
+            /*
+             * the file already exist, try to open it
+             */
+            fd = open(path.c_str(), O_RDONLY);
+        } else {
+            /*
+             * failed to create, do not care why
+             */
+            return 0;
+        }
+    } else {
+        if (0 != flock(fd, LOCK_EX)) {
+            /*
+             * failed to lock
+             */
+            close(fd);
+            return index;
+        }
+
+        /*
+         * created file, initialize it with 0
+         */
+        write(fd, &index, sizeof(index));
+        flock(fd, LOCK_UN);
+        close(fd);
+        return index;
+    }
+
+    /*
+     * the file exist, read it.
+     */
+    if (fd >= 0) {
+        if (0 != flock(fd, LOCK_SH)) {
+            /*
+             * failed to lock
+             */
+            close(fd);
+            return index;
+        }
+
+        if (sizeof(index) != read(fd, &index, sizeof(index))) {
+            /*
+             * failed to read, do not care why
+             */
+            index = 0;
+        }
+
+        flock(fd, LOCK_UN);
+        close(fd);
+    }
+
+    return index;
+}
+
+static void SetInitNamenodeIndex(const std::string &id, uint32_t index) {
+    std::string path = "/tmp/";
+    path += id;
+    int fd;
+    /*
+     * try open the file for write
+     */
+    fd = open(path.c_str(), O_WRONLY);
+
+    if (fd > 0) {
+        if (0 != flock(fd, LOCK_EX)) {
+            /*
+             * failed to lock
+             */
+            close(fd);
+            return;
+        }
+
+        write(fd, &index, sizeof(index));
+        flock(fd, LOCK_UN);
+        close(fd);
+    }
+}
+
+NamenodeProxy::NamenodeProxy(const std::vector<NamenodeInfo> &namenodeInfos,
+                             const std::string &tokenService,
+                             const SessionConfig &c, const RpcAuth &a) :
+    clusterid(tokenService), currentNamenode(0) {
+    if (namenodeInfos.size() == 1) {
+        enableNamenodeHA = false;
+        maxNamenodeHARetry = 0;
+    } else {
+        enableNamenodeHA = true;
+        maxNamenodeHARetry = c.getRpcMaxHaRetry();
+    }
+
+    for (size_t i = 0; i < namenodeInfos.size(); ++i) {
+        std::vector<std::string> nninfo = StringSplit(namenodeInfos[i].getRpcAddr(), ":");
+
+        if (nninfo.size() != 2) {
+            THROW(InvalidParameter, "Cannot create namenode proxy, %s does not contain host or port",
+                  namenodeInfos[i].getRpcAddr().c_str());
+        }
+
+        namenodes.push_back(
+            shared_ptr<Namenode>(
+                new NamenodeImpl(nninfo[0].c_str(), nninfo[1].c_str(), clusterid, c, a)));
+    }
+
+    if (enableNamenodeHA) {
+        currentNamenode = GetInitNamenodeIndex(clusterid) % namenodeInfos.size();
+    }
+}
+
+NamenodeProxy::~NamenodeProxy() {
+}
+
+shared_ptr<Namenode> NamenodeProxy::getActiveNamenode(uint32_t &oldValue) {
+    lock_guard<mutex> lock(mut);
+
+    if (namenodes.empty()) {
+        THROW(HdfsFileSystemClosed, "NamenodeProxy is closed.");
+    }
+
+    oldValue = currentNamenode;
+    return namenodes[currentNamenode % namenodes.size()];
+}
+
+void NamenodeProxy::failoverToNextNamenode(uint32_t oldValue) {
+    lock_guard<mutex> lock(mut);
+
+    if (oldValue != currentNamenode) {
+        //already failover in another thread.
+        return;
+    }
+
+    ++currentNamenode;
+    currentNamenode = currentNamenode % namenodes.size();
+    SetInitNamenodeIndex(clusterid, currentNamenode);
+}
+
+static void HandleHdfsFailoverException(const HdfsFailoverException &e) {
+    try {
+        rethrow_if_nested(e);
+    } catch (...) {
+        NESTED_THROW(hdfs::HdfsRpcException, "%s", e.what());
+    }
+
+    //should not reach here
+    abort();
+}
+
+#define NAMENODE_HA_RETRY_BEGIN() \
+    do { \
+        int __count = 0; \
+        do { \
+            uint32_t __oldValue = 0; \
+            shared_ptr<Namenode> namenode =  getActiveNamenode(__oldValue); \
+            try { \
+                (void)0
+
+#define NAMENODE_HA_RETRY_END() \
+    break; \
+    } catch (const NameNodeStandbyException &e) { \
+        if (!enableNamenodeHA || __count++ > maxNamenodeHARetry) { \
+            throw; \
+        } \
+    } catch (const HdfsFailoverException &e) { \
+        if (!enableNamenodeHA || __count++ > maxNamenodeHARetry) { \
+            HandleHdfsFailoverException(e); \
+        } \
+    } \
+    failoverToNextNamenode(__oldValue); \
+    LOG(WARNING, "NamenodeProxy: Failover to another Namenode."); \
+    } while (true); \
+    } while (0)
+
+void NamenodeProxy::getBlockLocations(const std::string &src, int64_t offset,
+        int64_t length, LocatedBlocks &lbs) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->getBlockLocations(src, offset, length, lbs);
+    NAMENODE_HA_RETRY_END();
+}
+
+void NamenodeProxy::create(const std::string &src, const Permission &masked,
+        const std::string &clientName, int flag, bool createParent,
+        short replication, int64_t blockSize) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->create(src, masked, clientName, flag, createParent, replication, blockSize);
+    NAMENODE_HA_RETRY_END();
+}
+
+shared_ptr<LocatedBlock> NamenodeProxy::append(const std::string &src,
+        const std::string &clientName) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->append(src, clientName);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return shared_ptr<LocatedBlock>();
+}
+
+bool NamenodeProxy::setReplication(const std::string &src, short replication) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->setReplication(src, replication);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+void NamenodeProxy::setPermission(const std::string &src,
+        const Permission &permission) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->setPermission(src, permission);
+    NAMENODE_HA_RETRY_END();
+}
+
+void NamenodeProxy::setOwner(const std::string &src,
+        const std::string &username, const std::string &groupname) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->setOwner(src, username, groupname);
+    NAMENODE_HA_RETRY_END();
+}
+
+void NamenodeProxy::abandonBlock(const ExtendedBlock &b,
+        const std::string &src, const std::string &holder) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->abandonBlock(b, src, holder);
+    NAMENODE_HA_RETRY_END();
+}
+
+shared_ptr<LocatedBlock> NamenodeProxy::addBlock(const std::string &src,
+        const std::string &clientName, const ExtendedBlock * previous,
+        const std::vector<DatanodeInfo> &excludeNodes) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->addBlock(src, clientName, previous, excludeNodes);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return shared_ptr<LocatedBlock>();
+}
+
+shared_ptr<LocatedBlock> NamenodeProxy::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) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getAdditionalDatanode(src, blk, existings,
+                  storageIDs, excludes, numAdditionalNodes, clientName);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return shared_ptr<LocatedBlock>();
+}
+
+bool NamenodeProxy::complete(const std::string &src,
+                  const std::string &clientName, const ExtendedBlock *last) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->complete(src, clientName, last);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+/*void NamenodeProxy::reportBadBlocks(const std::vector<LocatedBlock> &blocks) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->reportBadBlocks(blocks);
+    NAMENODE_HA_RETRY_END();
+}*/
+
+bool NamenodeProxy::rename(const std::string &src, const std::string &dst) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->rename(src, dst);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+/*
+void NamenodeProxy::concat(const std::string &trg,
+                           const std::vector<std::string> &srcs) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->concat(trg, srcs);
+    NAMENODE_HA_RETRY_END();
+}
+*/
+
+bool NamenodeProxy::deleteFile(const std::string &src, bool recursive) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->deleteFile(src, recursive);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+bool NamenodeProxy::mkdirs(const std::string &src, const Permission &masked,
+          bool createParent) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->mkdirs(src, masked, createParent);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+bool NamenodeProxy::getListing(const std::string &src,
+          const std::string &startAfter, bool needLocation,
+          std::vector<FileStatus> &dl) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getListing(src, startAfter, needLocation, dl);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+void NamenodeProxy::renewLease(const std::string &clientName) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->renewLease(clientName);
+    NAMENODE_HA_RETRY_END();
+}
+
+bool NamenodeProxy::recoverLease(const std::string &src,
+                                 const std::string &clientName) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->recoverLease(src, clientName);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return false;
+}
+
+std::vector<int64_t> NamenodeProxy::getFsStats() {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getFsStats();
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return std::vector<int64_t>();
+}
+
+/*void NamenodeProxy::metaSave(const std::string &filename) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->metaSave(filename);
+    NAMENODE_HA_RETRY_END();
+}*/
+
+FileStatus NamenodeProxy::getFileInfo(const std::string &src) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getFileInfo(src);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return FileStatus();
+}
+
+/*FileStatus NamenodeProxy::getFileLinkInfo(const std::string &src) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getFileLinkInfo(src);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return FileStatus();
+}*/
+
+/*ContentSummary NamenodeProxy::getContentSummary(const std::string &path) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getContentSummary(path);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return ContentSummary();
+}*/
+
+/*void NamenodeProxy::setQuota(const std::string &path, int64_t namespaceQuota,
+                             int64_t diskspaceQuota) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->setQuota(path, namespaceQuota, diskspaceQuota);
+    NAMENODE_HA_RETRY_END();
+}*/
+
+void NamenodeProxy::fsync(const std::string &src, const std::string &client) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->fsync(src, client);
+    NAMENODE_HA_RETRY_END();
+}
+
+void NamenodeProxy::setTimes(const std::string &src, int64_t mtime,
+                             int64_t atime) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->setTimes(src, mtime, atime);
+    NAMENODE_HA_RETRY_END();
+}
+
+/*void NamenodeProxy::createSymlink(const std::string &target,
+                const std::string &link, const Permission &dirPerm,
+                bool createParent) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->createSymlink(target, link, dirPerm, createParent);
+    NAMENODE_HA_RETRY_END();
+}*/
+
+/*std::string NamenodeProxy::getLinkTarget(const std::string &path) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getLinkTarget(path);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return "";
+}*/
+
+shared_ptr<LocatedBlock> NamenodeProxy::updateBlockForPipeline(
+          const ExtendedBlock &block, const std::string &clientName) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->updateBlockForPipeline(block, clientName);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return shared_ptr<LocatedBlock>();
+}
+
+void NamenodeProxy::updatePipeline(const std::string &clientName,
+          const ExtendedBlock &oldBlock, const ExtendedBlock &newBlock,
+          const std::vector<DatanodeInfo> &newNodes,
+          const std::vector<std::string> &storageIDs) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->updatePipeline(clientName, oldBlock, newBlock,
+                             newNodes, storageIDs);
+    NAMENODE_HA_RETRY_END();
+}
+
+Token NamenodeProxy::getDelegationToken(const std::string &renewer) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->getDelegationToken(renewer);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return Token();
+}
+
+int64_t NamenodeProxy::renewDelegationToken(const Token &token) {
+    NAMENODE_HA_RETRY_BEGIN();
+    return namenode->renewDelegationToken(token);
+    NAMENODE_HA_RETRY_END();
+    assert(!"should not reach here");
+    return 0;
+}
+
+void NamenodeProxy::cancelDelegationToken(const Token &token) {
+    NAMENODE_HA_RETRY_BEGIN();
+    namenode->cancelDelegationToken(token);
+    NAMENODE_HA_RETRY_END();
+}
+
+void NamenodeProxy::close() {
+    lock_guard<mutex> lock(mut);
+    namenodes.clear();
+}
+
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h
new file mode 100644
index 0000000..2cbd460
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h
@@ -0,0 +1,149 @@
+/**
+ * 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_SERVER_NAMENODEPROXY_H_
+#define _HDFS_LIBHDFS3_SERVER_NAMENODEPROXY_H_
+
+#include "Namenode.h"
+#include "NamenodeInfo.h"
+#include "SharedPtr.h"
+#include "Thread.h"
+
+namespace hdfs {
+namespace internal {
+
+class NamenodeProxy: public Namenode {
+public:
+    NamenodeProxy(const std::vector<NamenodeInfo> &namenodeInfos,
+            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);
+
+    void create(const std::string &src, const Permission &masked,
+            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);
+
+    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);
+
+    void abandonBlock(const ExtendedBlock &b, const std::string &src,
+            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> 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);
+
+    void reportBadBlocks(const std::vector<LocatedBlock> &blocks);
+
+    bool rename(const std::string &src, const std::string &dst);
+
+    void concat(const std::string &trg, const std::vector<std::string> &srcs);
+
+    /*void rename2(const std::string &src, const std::string &dst)
+     throw (AccessControlException, DSQuotaExceededException,
+     FileAlreadyExistsException, FileNotFoundException,
+     NSQuotaExceededException, ParentNotDirectoryException,
+     SafeModeException, UnresolvedLinkException, HdfsIOException) ;*/
+
+    bool deleteFile(const std::string &src, bool recursive);
+
+    bool mkdirs(const std::string &src, const Permission &masked,
+            bool createParent);
+
+    bool getListing(const std::string &src, const std::string &startAfter,
+            bool needLocation, std::vector<FileStatus> &dl);
+
+    void renewLease(const std::string &clientName);
+
+    bool recoverLease(const std::string &src, const std::string &clientName);
+
+    std::vector<int64_t> getFsStats();
+
+    void metaSave(const std::string &filename);
+
+    FileStatus getFileInfo(const std::string &src);
+
+    FileStatus getFileLinkInfo(const std::string &src);
+
+    void setQuota(const std::string &path, int64_t namespaceQuota,
+            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);
+
+    std::string getLinkTarget(const std::string &path);
+
+    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);
+
+    Token getDelegationToken(const std::string &renewer);
+
+    int64_t renewDelegationToken(const Token &token);
+
+    void cancelDelegationToken(const Token &token);
+
+    void close();
+
+private:
+    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;
+    uint32_t currentNamenode;
+};
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_NAMENODEPROXY_H_ */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/RpcHelper.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/RpcHelper.h
new file mode 100644
index 0000000..2183c55
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/RpcHelper.h
@@ -0,0 +1,290 @@
+/**
+ * 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_SERVER_RPCHELPER_H_
+#define _HDFS_LIBHDFS3_SERVER_RPCHELPER_H_
+
+#include "ClientDatanodeProtocol.pb.h"
+#include "ClientNamenodeProtocol.pb.h"
+#include "DatanodeInfo.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "ExtendedBlock.h"
+#include "LocatedBlock.h"
+#include "LocatedBlocks.h"
+#include "StackPrinter.h"
+#include "client/FileStatus.h"
+#include "client/Permission.h"
+#include "hdfs.pb.h"
+
+#include <algorithm>
+#include <cassert>
+
+using namespace google::protobuf;
+
+namespace hdfs {
+namespace internal {
+
+class Nothing {
+};
+
+template < typename T1 = Nothing, typename T2 = Nothing, typename T3 = Nothing,
+         typename T4 = Nothing, typename T5 = Nothing, typename T6 = Nothing,
+         typename T7 = Nothing, typename T8 = Nothing, typename T9 = Nothing,
+         typename T10 = Nothing, typename T11 = Nothing  >
+class UnWrapper: public UnWrapper<T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, Nothing> {
+private:
+    typedef UnWrapper<T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, Nothing> BaseType;
+
+public:
+    UnWrapper(const HdfsRpcServerException &e) :
+        BaseType(e), e(e) {
+    }
+
+    void ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE unwrap(const char *file,
+            int line) {
+        if (e.getErrClass() == T1::ReflexName) {
+#ifdef NEED_BOOST
+            boost::throw_exception(T1(e.getErrMsg(), SkipPathPrefix(file), line, PrintStack(1, STACK_DEPTH).c_str()));
+#else
+            throw T1(e.getErrMsg(), SkipPathPrefix(file), line, PrintStack(1, STACK_DEPTH).c_str());
+#endif
+        } else {
+            BaseType::unwrap(file, line);
+        }
+    }
+private:
+    const HdfsRpcServerException &e;
+};
+
+template<>
+class UnWrapper < Nothing, Nothing, Nothing, Nothing, Nothing, Nothing, Nothing,
+        Nothing, Nothing, Nothing, Nothing > {
+public:
+    UnWrapper(const HdfsRpcServerException &e) :
+        e(e) {
+    }
+    void ATTRIBUTE_NORETURN ATTRIBUTE_NOINLINE unwrap(const char *file,
+            int line) {
+        THROW(HdfsIOException,
+              "Unexpected exception: when unwrap the rpc remote exception \"%s\", %s in %s: %d",
+              e.getErrClass().c_str(), e.getErrMsg().c_str(), file, line);
+    }
+private:
+    const HdfsRpcServerException &e;
+};
+
+static inline void Convert(ExtendedBlock &eb,
+                const hadoop::hdfs::ExtendedBlockProto &proto) {
+    eb.setBlockId(proto.blockid());
+    eb.setGenerationStamp(proto.generationstamp());
+    eb.setNumBytes(proto.numbytes());
+    eb.setPoolId(proto.poolid());
+}
+
+static inline void Convert(Token &token,
+                const hadoop::common::TokenProto &proto) {
+    token.setIdentifier(proto.identifier());
+    token.setKind(proto.kind());
+    token.setPassword(proto.password());
+    token.setService(proto.service());
+}
+
+static inline void Convert(DatanodeInfo &node,
+                const hadoop::hdfs::DatanodeInfoProto &proto) {
+    const hadoop::hdfs::DatanodeIDProto &idProto = proto.id();
+    node.setHostName(idProto.hostname());
+    node.setInfoPort(idProto.infoport());
+    node.setIpAddr(idProto.ipaddr());
+    node.setIpcPort(idProto.ipcport());
+    node.setDatanodeId(idProto.datanodeuuid());
+    node.setXferPort(idProto.xferport());
+    node.setLocation(proto.location());
+}
+
+static inline shared_ptr<LocatedBlock> Convert(
+                const hadoop::hdfs::LocatedBlockProto &proto) {
+    Token token;
+    shared_ptr<LocatedBlock> lb(new LocatedBlock);
+    Convert(token, proto.blocktoken());
+    lb->setToken(token);
+    std::vector<DatanodeInfo> &nodes = lb->mutableLocations();
+    nodes.resize(proto.locs_size());
+
+    for (int i = 0 ; i < proto.locs_size(); ++i) {
+        Convert(nodes[i], proto.locs(i));
+    }
+
+    if (proto.storagetypes_size() > 0) {
+        assert(proto.storagetypes_size() == proto.locs_size());
+        std::vector<std::string> &storageIDs = lb->mutableStorageIDs();
+        storageIDs.resize(proto.storagetypes_size());
+
+        for (int i = 0; i < proto.storagetypes_size(); ++i) {
+            storageIDs[i] = proto.storageids(i);
+        }
+    }
+
+    Convert(*lb, proto.b());
+    lb->setOffset(proto.offset());
+    lb->setCorrupt(proto.corrupt());
+    return lb;
+}
+
+static inline void Convert(LocatedBlocks &lbs,
+                const hadoop::hdfs::LocatedBlocksProto &proto) {
+    shared_ptr<LocatedBlock> lb;
+    lbs.setFileLength(proto.filelength());
+    lbs.setIsLastBlockComplete(proto.islastblockcomplete());
+    lbs.setUnderConstruction(proto.underconstruction());
+
+    if (proto.has_lastblock()) {
+        lb = Convert(proto.lastblock());
+        lbs.setLastBlock(lb);
+    }
+
+    std::vector<LocatedBlock> &blocks = lbs.getBlocks();
+    blocks.resize(proto.blocks_size());
+
+    for (int i = 0; i < proto.blocks_size(); ++i) {
+        blocks[i] = *Convert(proto.blocks(i));
+    }
+
+    std::sort(blocks.begin(), blocks.end(), std::less<LocatedBlock>());
+}
+
+static inline void Convert(FileStatus &fs,
+                const hadoop::hdfs::HdfsFileStatusProto &proto) {
+    fs.setAccessTime(proto.access_time());
+    fs.setBlocksize(proto.blocksize());
+    fs.setGroup(proto.group().c_str());
+    fs.setLength(proto.length());
+    fs.setModificationTime(proto.modification_time());
+    fs.setOwner(proto.owner().c_str());
+    fs.setPath(proto.path().c_str());
+    fs.setReplication(proto.block_replication());
+    fs.setSymlink(proto.symlink().c_str());
+    fs.setPermission(Permission(proto.permission().perm()));
+    fs.setIsdir(proto.filetype() == hadoop::hdfs::HdfsFileStatusProto::IS_DIR);
+}
+
+static inline void Convert(std::vector<FileStatus> &dl,
+                const hadoop::hdfs::DirectoryListingProto &proto) {
+    RepeatedPtrField<hadoop::hdfs::HdfsFileStatusProto> ptrproto =
+          proto.partiallisting();
+
+    for (int i = 0; i < ptrproto.size(); i++) {
+        FileStatus fileStatus;
+        Convert(fileStatus, ptrproto.Get(i));
+        dl.push_back(fileStatus);
+    }
+}
+
+static inline Token Convert(const hadoop::common::TokenProto &proto) {
+    Token retval;
+    retval.setIdentifier(proto.identifier());
+    retval.setKind(proto.kind());
+    retval.setPassword(proto.password());
+    return retval;
+}
+
+/*static inline void Convert(ContentSummary &contentSummary, const ContentSummaryProto &proto) {
+    contentSummary.setDirectoryCount(proto.directorycount());
+    contentSummary.setFileCount(proto.filecount());
+    contentSummary.setLength(proto.length());
+    contentSummary.setQuota(proto.quota());
+    contentSummary.setSpaceConsumed(proto.spaceconsumed());
+    contentSummary.setSpaceQuota(proto.spacequota());
+}*/
+
+static inline void Build(const Token &token,
+                hadoop::common::TokenProto *proto) {
+    proto->set_identifier(token.getIdentifier());
+    proto->set_kind(token.getKind());
+    proto->set_password(token.getPassword());
+    proto->set_service(token.getService());
+}
+
+static inline void Build(const Permission &p,
+                hadoop::hdfs::FsPermissionProto *proto) {
+    proto->set_perm(p.toShort());
+}
+
+static inline void Build(const DatanodeInfo &dn,
+                hadoop::hdfs::DatanodeIDProto *proto) {
+    proto->set_hostname(dn.getHostName());
+    proto->set_infoport(dn.getInfoPort());
+    proto->set_ipaddr(dn.getIpAddr());
+    proto->set_ipcport(dn.getIpcPort());
+    proto->set_datanodeuuid(dn.getDatanodeId());
+    proto->set_xferport(dn.getXferPort());
+}
+
+static inline void Build(const std::vector<DatanodeInfo> &dns,
+                RepeatedPtrField<hadoop::hdfs::DatanodeInfoProto> *proto) {
+    for (size_t i = 0; i < dns.size(); ++i) {
+        hadoop::hdfs::DatanodeInfoProto *p = proto->Add();
+        Build(dns[i], p->mutable_id());
+        p->set_location(dns[i].getLocation());
+    }
+}
+
+static inline void Build(const ExtendedBlock &eb,
+                hadoop::hdfs::ExtendedBlockProto *proto) {
+    proto->set_blockid(eb.getBlockId());
+    proto->set_generationstamp(eb.getGenerationStamp());
+    proto->set_numbytes(eb.getNumBytes());
+    proto->set_poolid(eb.getPoolId());
+}
+
+static inline void Build(LocatedBlock &b,
+                hadoop::hdfs::LocatedBlockProto *proto) {
+    proto->set_corrupt(b.isCorrupt());
+    proto->set_offset(b.getOffset());
+    Build(b, proto->mutable_b());
+    Build(b.getLocations(), proto->mutable_locs());
+}
+
+/*static inline void Build(const std::vector<LocatedBlock> &blocks,
+                         RepeatedPtrField<LocatedBlockProto> *proto) {
+    for (size_t i = 0; i < blocks.size(); ++i) {
+        LocatedBlockProto *p = proto->Add();
+        p->set_corrupt(blocks[i].isCorrupt());
+        p->set_offset(blocks[i].getOffset());
+        Build(blocks[i], p->mutable_b());
+    }
+}*/
+
+static inline void Build(const std::vector<std::string> &srcs,
+                         RepeatedPtrField<std::string> *proto) {
+    for (size_t i = 0; i < srcs.size(); ++i) {
+        proto->Add()->assign(srcs[i]);
+    }
+}
+
+static inline void Build(const std::vector<DatanodeInfo> &dns,
+                RepeatedPtrField<hadoop::hdfs::DatanodeIDProto> *proto) {
+    for (size_t i = 0; i < dns.size(); ++i) {
+        Build(dns[i], proto->Add());
+    }
+}
+
+}
+}
+
+#endif /* _HDFS_LIBHDFS3_SERVER_RPCHELPER_H_ */