You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cm...@apache.org on 2015/02/04 21:45:26 UTC

hadoop git commit: HDFS-7018. Implement hdfs.h interface in libhdfs3 (wangzw via cmccabe)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-6994 650c3c6b7 -> af17a556c


HDFS-7018. Implement hdfs.h interface in libhdfs3 (wangzw via cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/af17a556
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/af17a556
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/af17a556

Branch: refs/heads/HDFS-6994
Commit: af17a556ce5d2e652d8c5aac99554617eb8143f4
Parents: 650c3c6
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Feb 4 12:44:49 2015 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Feb 4 12:44:49 2015 -0800

----------------------------------------------------------------------
 .../src/contrib/libhdfs3/CMakeLists.txt         |  2 +
 .../src/contrib/libhdfs3/src/CMakeLists.txt     |  4 ++
 .../libhdfs3/src/GenerateProtobufs.cmake        | 54 +++++++++---------
 .../libhdfs3/src/client/DirectoryIterator.h     |  2 +-
 .../libhdfs3/src/client/FileSystemImpl.cc       | 26 ---------
 .../contrib/libhdfs3/src/client/InputStream.cc  | 59 +++++++++-----------
 .../contrib/libhdfs3/src/client/InputStream.h   | 24 ++++----
 .../src/contrib/libhdfs3/src/common/Atoi.cc     |  6 ++
 .../src/contrib/libhdfs3/src/common/Atoi.h      |  8 +--
 .../src/contrib/libhdfs3/src/common/Status.h    | 12 ++--
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.c  |  8 +++
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.h  |  9 +++
 12 files changed, 102 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt
index 4bf220e..7d5cef9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/CMakeLists.txt
@@ -39,6 +39,8 @@ FIND_PACKAGE(KERBEROS REQUIRED)
 FIND_PACKAGE(GSasl REQUIRED)
 FIND_PACKAGE(EXPAT REQUIRED)
 
+GET_FILENAME_COMPONENT(HADOOP_TOP_DIR "${PROJECT_SOURCE_DIR}/../../../../../" REALPATH)
+
 ADD_SUBDIRECTORY(mock)
 ADD_SUBDIRECTORY(src)
 ADD_SUBDIRECTORY(gtest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/CMakeLists.txt
----------------------------------------------------------------------
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 f7d4a0e..ad0e0a2 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
@@ -53,7 +53,11 @@ SET(libhdfs3_PROTO_FILES ${libhdfs3_PROTO_FILES} PARENT_SCOPE)
 INCLUDE(GenerateProtobufs.cmake)
 INCLUDE_DIRECTORIES("${CMAKE_BINARY_DIR}")
 
+SET(LIBHDFS_SRC_DIR ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/)
+INCLUDE_DIRECTORIES(${LIBHDFS_SRC_DIR})
+
 SET(HEADER 
+    ${LIBHDFS_SRC_DIR}/hdfs.h
     client/BlockLocation.h
     client/DirectoryIterator.h
     client/FileStatus.h

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
----------------------------------------------------------------------
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
index 8cad7b9..f0a9554 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/GenerateProtobufs.cmake
@@ -26,38 +26,36 @@ function(COPY_IF_CHANGED TARGET)
     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
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/GetUserMappingsProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/IpcConnectionContext.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/RefreshCallQueueProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/RefreshUserMappingsProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
+    ${HADOOP_TOP_DIR}/hadoop-common-project/hadoop-common/src/main/proto/Security.proto
+    ${HADOOP_TOP_DIR}/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
-    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
-    ${R}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
+    #${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HAZKInfo.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/acl.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
+    #${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
+    ${HADOOP_TOP_DIR}/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/inotify.proto
 )
 
 AUTO_SOURCES(PB_SOURCES "*.proto" "RECURSE" "${CMAKE_BINARY_DIR}")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h
index 47708db..1df627a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/DirectoryIterator.h
@@ -48,7 +48,7 @@ private:
     std::string startAfter;
     std::vector<FileStatus> lists;
 
-    friend hdfs::internal::FileSystemImpl;
+    friend class hdfs::internal::FileSystemImpl;
 };
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
index a51b62a..74f3489 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/FileSystemImpl.cc
@@ -533,32 +533,6 @@ FileSystemStats FileSystemImpl::getFsStats() {
     return FileSystemStats(retval[0], retval[1], retval[2]);
 }
 
-static std::string ConstructTempFilePath(const std::string &path,
-                                         const std::string clientName) {
-    std::stringstream ss;
-    srand((unsigned int)time(NULL));
-    static atomic<uint32_t> count(0);
-    std::vector<std::string> components = StringSplit(path, "/");
-    ss << '/';
-
-    for (size_t i = components.size(); i > 0; --i) {
-        if (!components[i - 1].empty()) {
-            components[i - 1].clear();
-            break;
-        }
-    }
-
-    for (size_t i = 0; i < components.size(); ++i) {
-        if (!components[i].empty()) {
-            ss << components[i] << '/';
-        }
-    }
-
-    ss << "._client_" << clientName << "_random_" << rand() << "_count_"
-       << ++count << "_tid_" << pthread_self() << "_TRUNCATE_TMP";
-    return ss.str();
-}
-
 std::string FileSystemImpl::getDelegationToken(const char *renewer) {
     if (!nn) {
         THROW(HdfsIOException, "FileSystemImpl: not connected.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc
index 3168fe4..aa6a8e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.cc
@@ -35,89 +35,82 @@ InputStream::~InputStream() {
 
 Status InputStream::open(FileSystem &fs, const std::string &path,
                          bool verifyChecksum) {
-    if (!fs.impl) {
-        THROW(HdfsIOException, "FileSystem: not connected.");
-    }
+    CHECK_PARAMETER(fs.impl, EIO, "FileSystem: not connected.");
 
     try {
         impl->open(fs.impl, path.c_str(), verifyChecksum);
     } catch (...) {
-        return lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return lastError = Status::OK();
+    return Status::OK();
 }
 
-int32_t InputStream::read(char *buf, int32_t size) {
-    int32_t retval = -1;
+Status InputStream::read(char *buf, int32_t size, int32_t *done) {
+    CHECK_PARAMETER(NULL != done, EINVAL, "invalid parameter \"output\"");
 
     try {
-        retval = impl->read(buf, size);
-        lastError = Status::OK();
+        *done = impl->read(buf, size);
+    } catch (const HdfsEndOfStream &e) {
+        *done = 0;
     } catch (...) {
-        lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return retval;
+    return Status::OK();
 }
 
 Status InputStream::readFully(char *buf, int64_t size) {
     try {
         impl->readFully(buf, size);
     } catch (...) {
-        return lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return lastError = Status::OK();
+    return Status::OK();
 }
 
-int64_t InputStream::available() {
-    int64_t retval = -1;
+Status InputStream::available(int64_t *output) {
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
 
     try {
-        retval = impl->available();
-        lastError = Status::OK();
+        *output = impl->available();
     } catch (...) {
-        lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return retval;
+    return Status::OK();
 }
 
 Status InputStream::seek(int64_t pos) {
     try {
         impl->seek(pos);
     } catch (...) {
-        return lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return lastError = Status::OK();
+    return Status::OK();
 }
 
-int64_t InputStream::tell() {
-    int64_t retval = -1;
+Status InputStream::tell(int64_t *output) {
+    CHECK_PARAMETER(NULL != output, EINVAL, "invalid parameter \"output\"");
 
     try {
-        retval = impl->tell();
-        lastError = Status::OK();
+        *output = impl->tell();
     } catch (...) {
-        lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return retval;
+    return Status::OK();
 }
 
 Status InputStream::close() {
     try {
         impl->close();
     } catch (...) {
-        return lastError = CreateStatusFromException(current_exception());
+        return CreateStatusFromException(current_exception());
     }
 
-    return lastError = Status::OK();
-}
-
-Status InputStream::getLastError() {
-    return lastError;
+    return Status::OK();
 }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/client/InputStream.h
----------------------------------------------------------------------
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
index bd7a5dc..d93e450 100644
--- 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
@@ -57,10 +57,11 @@ public:
      * 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, -1 on error.
+     * @param done the number of bytes filled in the buffer, it may less than
+     *             size.
+     * @return the result status of this operation
      */
-    int32_t read(char *buf, int32_t size);
+    Status read(char *buf, int32_t size, int32_t *done);
 
     /**
      * To read data from hdfs, block until get the given size of bytes.
@@ -72,9 +73,10 @@ public:
 
     /**
      * Get how many bytes can be read without blocking.
-     * @return The number of bytes can be read without blocking, -1 on error.
+     * @param output The number of bytes can be read without blocking.
+     * @return the result status of this operation
      */
-    int64_t available();
+    Status available(int64_t *output);
 
     /**
      * To move the file point to the given position.
@@ -85,9 +87,10 @@ public:
 
     /**
      * To get the current file point position.
-     * @return the position of current file pointer, -1 on error.
+     * @param the position of current file pointer.
+     * @return the result status of this operation
      */
-    int64_t tell();
+    Status tell(int64_t *output);
 
     /**
      * Close the stream.
@@ -95,18 +98,11 @@ public:
      */
     Status close();
 
-    /**
-     * Get the error status of the last operation.
-     * @return the error status of the last operation.
-     */
-    Status getLastError();
-
 private:
     InputStream(const InputStream &other);
     InputStream &operator=(InputStream &other);
 
     internal::InputStreamImpl *impl;
-    Status lastError;
 };
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc
index afc3de0..58786c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.cc
@@ -28,6 +28,9 @@
 using hdfs::Status;
 using std::ostringstream;
 
+namespace hdfs {
+namespace internal {
+
 Status StrToInt32(const char *str, int32_t *ret) {
     long retval;
     char *end = NULL;
@@ -106,3 +109,6 @@ Status StrToDouble(const char *str, double *ret) {
     *ret = retval;
     return Status::OK();
 }
+
+}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h
index 3ebb1b6..e9cf694 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Atoi.h
@@ -26,13 +26,13 @@
 namespace hdfs {
 namespace internal {
 
-  Status StrToInt32(const char *str, int32_t *ret);
+Status StrToInt32(const char *str, int32_t *ret);
 
-  Status StrToInt64(const char *str, int64_t *ret);
+Status StrToInt64(const char *str, int64_t *ret);
 
-  Status StrToBool(const char *str, bool *ret);
+Status StrToBool(const char *str, bool *ret);
 
-  Status StrToDouble(const char *str, double *ret);
+Status StrToDouble(const char *str, double *ret);
 
 }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
index 32dff20..4d8ca67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libhdfs3/src/common/Status.h
@@ -43,7 +43,7 @@ public:
         return code;
     }
 
-    const char *getErrorMsg() {
+    const char *getErrorMsg() const {
         return msg.c_str();
     }
 
@@ -58,11 +58,11 @@ private:
 }
 
 #define RETURN_NOT_OK(expr) \
-    do { \
-        Status s = expr; \
-        if (s.isError()) { \
-            return s; \
-        } \
+    do {                    \
+        Status s = expr;    \
+        if (s.isError()) {  \
+            return s;       \
+        }                   \
     } while (0);
 
 #endif /* _HDFS_LIBHDFS3_COMMON_STATUS_H_ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
index dc8f39d..909ae63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
@@ -88,6 +88,14 @@ struct hdfsExtendedFileInfo {
     int flags;
 };
 
+/**
+ * not implemented yet
+ */
+const char* hdfsGetLastError()
+{
+    return "unknown error";
+}
+
 int hdfsFileIsOpenForRead(hdfsFile file)
 {
     return (file->type == HDFS_STREAM_INPUT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af17a556/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
index 0625da3..512d2d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
@@ -72,6 +72,15 @@ extern  "C" {
     struct hadoopRzBuffer;
 
     /**
+     * Return error information of last failed operation.
+     *
+     * @return A not NULL const string pointer of last error information
+     *         in the caller thread. Successful operations do not clear
+     *         this message.
+     */
+    const char* hdfsGetLastError();
+
+    /**
      * Determine if a file is open for read.
      *
      * @param file     The HDFS file