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