Browse Source

HDFS-7012. Add hdfs native client RPC functionality (Zhanwei Wang via Colin P. McCabe)

Colin Patrick Mccabe 10 năm trước cách đây
mục cha
commit
4b2cc72fa0
75 tập tin đã thay đổi với 10648 bổ sung12 xóa
  1. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMake/Platform.cmake
  2. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/bootstrap
  3. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
  4. 66 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
  5. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/BlockReader.h
  6. 101 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocol.h
  7. 179 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.cc
  8. 120 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DataTransferProtocolSender.h
  9. 159 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileStatus.h
  10. 90 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h
  11. 191 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamImpl.h
  12. 96 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStreamInter.h
  13. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.cc
  14. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/KerberosName.h
  15. 314 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.cc
  16. 98 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/LocalBlockReader.h
  17. 147 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.cc
  18. 122 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Packet.h
  19. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.cc
  20. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/PacketHeader.h
  21. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.cc
  22. 215 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Permission.h
  23. 375 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.cc
  24. 100 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/RemoteBlockReader.h
  25. 172 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.cc
  26. 82 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/Token.h
  27. 72 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.cc
  28. 97 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/UserInfo.h
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/LruMap.h
  30. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/SharedPtr.h
  31. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/StackPrinter.h
  32. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/UnorderedMap.h
  33. 124 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.cc
  34. 128 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/BufferedSocketReader.h
  35. 154 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Socket.h
  36. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/Syscall.h
  37. 406 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.cc
  38. 172 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/network/TcpSocket.h
  39. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.cc
  40. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcAuth.h
  41. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcCall.h
  42. 876 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.cc
  43. 272 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannel.h
  44. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.cc
  45. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcChannelKey.h
  46. 188 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.cc
  47. 155 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcClient.h
  48. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.cc
  49. 146 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcConfig.h
  50. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.cc
  51. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcContentWrapper.h
  52. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.cc
  53. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcProtocolInfo.h
  54. 79 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.cc
  55. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcRemoteCall.h
  56. 32 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.cc
  57. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/RpcServerInfo.h
  58. 157 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.cc
  59. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/rpc/SaslClient.h
  60. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/BlockLocalPathInfo.h
  61. 99 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.cc
  62. 103 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Datanode.h
  63. 126 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/DatanodeInfo.h
  64. 95 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/ExtendedBlock.h
  65. 105 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlock.h
  66. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.cc
  67. 111 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/LocatedBlocks.h
  68. 770 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/Namenode.h
  69. 730 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.cc
  70. 222 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeImpl.h
  71. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.cc
  72. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeInfo.h
  73. 491 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.cc
  74. 149 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/NamenodeProxy.h
  75. 290 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/server/RpcHelper.h

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

@@ -28,8 +28,6 @@ IF(CMAKE_COMPILER_IS_GNUCXX)
     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)

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


+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt

@@ -34,23 +34,24 @@ CONFIGURE_FILE(platform.h.in platform.h)
 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 @@ INSTALL(TARGETS libhdfs3-static libhdfs3-shared
         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)

+ 66 - 0
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}")

+ 57 - 0
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_ */

+ 101 - 0
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_ */

+ 179 - 0
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());
+    }
+}
+
+}
+}
+

+ 120 - 0
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_ */

+ 159 - 0
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

+ 90 - 0
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_ */

+ 191 - 0
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_ */

+ 96 - 0
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_ */

+ 113 - 0
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]));
+}
+
+}
+}

+ 93 - 0
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_ */

+ 314 - 0
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());
+    }
+}
+
+}
+}

+ 98 - 0
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_ */

+ 147 - 0
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);
+}
+
+}
+}

+ 122 - 0
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_ */

+ 117 - 0
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));
+}
+
+}
+}

+ 61 - 0
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_ */

+ 38 - 0
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);
+}
+
+}

+ 215 - 0
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

+ 375 - 0
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());
+    }
+}
+
+}
+}

+ 100 - 0
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_ */

+ 172 - 0
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]));
+}
+
+}
+}

+ 82 - 0
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

+ 72 - 0
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]));
+}
+
+}
+}

+ 97 - 0
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_ */

+ 1 - 1
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>
 

+ 13 - 0
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 @@ using std::tr1::shared_ptr;
 }
 
 #endif
+#endif

+ 1 - 1
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
 

+ 16 - 1
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 @@ using std::tr1::unordered_map;
 }
 
 #endif
+#endif

+ 124 - 0
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);
+}
+
+}
+}

+ 128 - 0
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_ */

+ 154 - 0
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_ */

+ 56 - 0
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_ */

+ 406 - 0
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;
+    }
+}
+
+}
+}

+ 172 - 0
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_ */

+ 46 - 0
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]));
+}
+
+}
+}

+ 93 - 0
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_ */

+ 78 - 0
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_ */

+ 876 - 0
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()));
+            }
+        }
+    }
+}
+
+}
+}

+ 272 - 0
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_ */

+ 46 - 0
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]));
+}
+
+}
+}

+ 86 - 0
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_ */

+ 188 - 0
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;
+}
+
+}
+}

+ 155 - 0
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_ */

+ 36 - 0
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]));
+}
+
+}
+}

+ 146 - 0
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_ */

+ 56 - 0
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);
+    }
+}
+
+}
+}
+

+ 45 - 0
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_ */

+ 30 - 0
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]));
+}
+
+}
+}

+ 78 - 0
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_ */

+ 79 - 0
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;
+}
+
+}
+}
+

+ 113 - 0
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_ */

+ 32 - 0
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]));
+}
+
+}
+}

+ 78 - 0
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_ */

+ 157 - 0
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;
+}
+
+}
+}
+

+ 62 - 0
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_ */

+ 64 - 0
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_ */

+ 99 - 0
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__);
+    }
+}
+
+}
+}

+ 103 - 0
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_ */

+ 126 - 0
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_ */

+ 95 - 0
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_ */

+ 105 - 0
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_ */

+ 69 - 0
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();
+    }
+}
+
+}
+}

+ 111 - 0
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_ */

+ 770 - 0
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_ */

+ 730 - 0
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__);
+    }
+}
+
+}
+}

+ 222 - 0
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_ */

+ 59 - 0
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;
+}
+}

+ 59 - 0
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_ */

+ 491 - 0
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();
+}
+
+}
+}

+ 149 - 0
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_ */

+ 290 - 0
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_ */